[hive] branch master updated: HIVE-24666: Vectorized UDFToBoolean may unable to filter rows if input is string (Zhihua Deng, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 8215cf6 HIVE-24666: Vectorized UDFToBoolean may unable to filter rows if input is string (Zhihua Deng, reviewed by Gopal V) 8215cf6 is described below commit 8215cf6b0ccf96234837292d46d71030c90090a3 Author: dengzhhu653 AuthorDate: Tue Feb 16 13:28:21 2021 -0800 HIVE-24666: Vectorized UDFToBoolean may unable to filter rows if input is string (Zhihua Deng, reviewed by Gopal V) Signed-off-by: Gopal V --- .../hive/ql/exec/vector/VectorizationContext.java | 64 ++-- .../exec/vector/expressions/FuncStringToLong.java | 4 +- .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 15 + .../hive/ql/parse/type/TypeCheckProcFactory.java | 23 ++ .../expressions/TestVectorFilterExpressions.java | 55 +++ .../queries/clientpositive/vector_cast_filter.q| 34 ++ .../llap/annotate_stats_filter.q.out | 10 +- ..._empty_where.q.out => vector_cast_filter.q.out} | 426 ++--- .../clientpositive/llap/vector_empty_where.q.out | 20 +- 9 files changed, 462 insertions(+), 189 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java index 9958f66..4af7325 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java @@ -871,28 +871,11 @@ import com.google.common.annotations.VisibleForTesting; } else { // Ok, we need to convert. List exprAsList = Collections.singletonList(exprDesc); - - // First try our cast method that will handle a few special cases. - VectorExpression castToBooleanExpr = getCastToBoolean(exprAsList); - if (castToBooleanExpr == null) { - -// Ok, try the UDF. -castToBooleanExpr = getVectorExpressionForUdf(null, UDFToBoolean.class, exprAsList, -VectorExpressionDescriptor.Mode.PROJECTION, TypeInfoFactory.booleanTypeInfo); -if (castToBooleanExpr == null) { - throw new HiveException("Cannot vectorize converting expression " + - exprDesc.getExprString() + " to boolean"); -} + expr = getCastToBooleanExpression(exprAsList, VectorExpressionDescriptor.Mode.FILTER); + if (expr == null) { +throw new HiveException("Cannot vectorize converting expression " + +exprDesc.getExprString() + " to boolean"); } - - final int outputColumnNum = castToBooleanExpr.getOutputColumnNum(); - - expr = new SelectColumnIsTrue(outputColumnNum); - - expr.setChildExpressions(new VectorExpression[] {castToBooleanExpr}); - - expr.setInputTypeInfos(castToBooleanExpr.getOutputTypeInfo()); - expr.setInputDataTypePhysicalVariations(DataTypePhysicalVariation.NONE); } return expr; } @@ -1483,6 +1466,10 @@ import com.google.common.annotations.VisibleForTesting; // Boolean is purposely excluded. } + public static boolean isCastToBoolean(Class udfClass) { +return udfClass.equals(UDFToBoolean.class); + } + public static boolean isCastToFloatFamily(Class udfClass) { return udfClass.equals(UDFToDouble.class) || udfClass.equals(UDFToFloat.class); @@ -1598,7 +1585,11 @@ import com.google.common.annotations.VisibleForTesting; if (typeInfo.getCategory() != Category.PRIMITIVE) { throw new HiveException("Complex type constants (" + typeInfo.getCategory() + ") not supported for type name " + typeName); } - return new ConstantVectorExpression(outCol, typeInfo, true); + if (mode == VectorExpressionDescriptor.Mode.FILTER) { +return new FilterConstantBooleanVectorExpression(0); + } else { +return new ConstantVectorExpression(outCol, typeInfo, true); + } } // Boolean is special case. @@ -2992,8 +2983,8 @@ import com.google.common.annotations.VisibleForTesting; PrimitiveCategory integerPrimitiveCategory = getAnyIntegerPrimitiveCategoryFromUdfClass(cl); ve = getCastToLongExpression(childExpr, integerPrimitiveCategory); -} else if (cl.equals(UDFToBoolean.class)) { - ve = getCastToBoolean(childExpr); +} else if (isCastToBoolean(cl)) { + ve = getCastToBooleanExpression(childExpr, mode); } else if (isCastToFloatFamily(cl)) { ve = getCastToDoubleExpression(cl, childExpr, returnType); } @@ -3470,26 +3461,41 @@ import com.google.common.annotations.VisibleForTesting; return null; } - private VectorExpression getCastToBoolean(List childExpr) + private VectorExpression getCastToBooleanExpression(List childExpr, Ve
[hive] branch master updated: HIVE-23596: LLAP: Encode initial guaranteed task information in containerId (Mustafa Iman, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new b5e3bcd HIVE-23596: LLAP: Encode initial guaranteed task information in containerId (Mustafa Iman, reviewed by Gopal V) b5e3bcd is described below commit b5e3bcd575aa7f904ec0955793916d0b11f38215 Author: Mustafa Iman AuthorDate: Thu Jun 4 10:18:29 2020 -0700 HIVE-23596: LLAP: Encode initial guaranteed task information in containerId (Mustafa Iman, reviewed by Gopal V) Signed-off-by: Gopal V --- .../hive/llap/tezplugins/ContainerFactory.java | 41 -- .../hive/llap/tezplugins/LlapTaskCommunicator.java | 5 +-- .../llap/tezplugins/LlapTaskSchedulerService.java | 39 +--- .../tezplugins/TestLlapTaskSchedulerService.java | 27 ++ 4 files changed, 76 insertions(+), 36 deletions(-) diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/ContainerFactory.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/ContainerFactory.java index f1feec7..d340c22 100644 --- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/ContainerFactory.java +++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/ContainerFactory.java @@ -28,6 +28,29 @@ class ContainerFactory { final ApplicationAttemptId customAppAttemptId; AtomicLong nextId; + private static final int GUARANTEED_WIDTH = 1; + + private static final long GUARANTEED_BIT_MASK = (1L << GUARANTEED_WIDTH) - 1; + + /** + * This is a hack to pass initial guaranteed information from {@link LlapTaskSchedulerService} + * to {@link LlapTaskCommunicator}. Otherwise, we ended up synchronizing communication and + * scheduling. This workaround can be removed after TEZ-4192 and HIVE-23589 are merged. + * + * Note: This method is reliable only for initial allocation of the task. Guaranteed status + * may change via separate requests later. Therefore, do not rely on this method other + * than creating initial submit work request. + * + * Even containerId -> guaranteed + * Odd containerId -> not guaranteed + * + * @param containerId + * @return {@code true} if the task associated with container was guaranteed initially. + */ + public static boolean isContainerInitializedAsGuaranteed(ContainerId containerId) { +return (containerId.getContainerId() & GUARANTEED_BIT_MASK) == 0; + } + public ContainerFactory(ApplicationAttemptId appAttemptId, long appIdLong) { this.nextId = new AtomicLong(1); ApplicationId appId = @@ -37,9 +60,9 @@ class ContainerFactory { } public Container createContainer(Resource capability, Priority priority, String hostname, - int port, String nodeHttpAddress) { + int port, String nodeHttpAddress, boolean isGuaranteed) { ContainerId containerId = -ContainerId.newContainerId(customAppAttemptId, nextId.getAndIncrement()); +ContainerId.newContainerId(customAppAttemptId, nextContainerId(isGuaranteed)); NodeId nodeId = NodeId.newInstance(hostname, port); Container container = @@ -47,4 +70,18 @@ class ContainerFactory { return container; } + + /** + * See {@link #isContainerInitializedAsGuaranteed(ContainerId)} + * @param isInitialGuaranteed + * @return + */ + private long nextContainerId(boolean isInitialGuaranteed) { +long candidate = nextId.getAndIncrement(); +candidate <<= GUARANTEED_WIDTH; +if (!isInitialGuaranteed) { + candidate |= 1; +} +return candidate; + } } diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java index 36a2d6b..73ae191 100644 --- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java +++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java @@ -858,10 +858,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl { taskSpec, currentQueryIdentifierProto, getTokenIdentifier(), user, hiveQueryId)).build()); // Don't call builder.setWorkSpecSignature() - Tez doesn't sign fragments builder.setFragmentRuntimeInfo(fragmentRuntimeInfo); -if (scheduler != null) { // May be null in tests - // TODO: see javadoc - builder.setIsGuaranteed(scheduler.isInitialGuaranteed(taskSpec.getTaskAttemptID())); -} + builder.setIsGuaranteed(ContainerFactory.isContainerInitializedAsGuaranteed(containerId)); return builder.build(); } diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java index 4e5e43c..d4cc7a3 100644 --- a/llap-tez
[hive] branch master updated: HIVE-23606: LLAP: Delay In DirectByteBuffer Clean Up For EncodedReaderImpl (Syed Shameerur Rahman, via Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 79ea2ef HIVE-23606: LLAP: Delay In DirectByteBuffer Clean Up For EncodedReaderImpl (Syed Shameerur Rahman, via Gopal V) 79ea2ef is described below commit 79ea2efede98b22474876c2667a5834452a2093b Author: Syed Shameerur Rahman AuthorDate: Thu Jun 4 10:56:30 2020 +0530 HIVE-23606: LLAP: Delay In DirectByteBuffer Clean Up For EncodedReaderImpl (Syed Shameerur Rahman, via Gopal V) Signed-off-by: Gopal V --- .../java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java | 1 - 1 file changed, 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java index 1f5a9d7..5c5ff4f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java @@ -630,7 +630,6 @@ class EncodedReaderImpl implements EncodedReader { } dataReader.readFileData(toRead.next, stripeOffset, cacheWrapper.getAllocator().isDirectAlloc()); -toRelease = new IdentityHashMap<>(); DiskRangeList drl = toRead.next; while (drl != null) { if (drl instanceof BufferChunk) {
[hive] branch master updated: HIVE-23178: Add Tez Total Order Partitioner (addendum, whitespace fixes)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new ece43ea HIVE-23178: Add Tez Total Order Partitioner (addendum, whitespace fixes) ece43ea is described below commit ece43eaa369d7224c47add8e2c42891c73408cc4 Author: Gopal V AuthorDate: Mon May 18 10:19:09 2020 -0700 HIVE-23178: Add Tez Total Order Partitioner (addendum, whitespace fixes) Signed-off-by: Gopal V --- .../hive/ql/exec/tez/TezTotalOrderPartitioner.java | 82 +++--- .../ql/exec/tez/TestTezTotalOrderPartitioner.java | 315 +++-- 2 files changed, 201 insertions(+), 196 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTotalOrderPartitioner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTotalOrderPartitioner.java index c66d964..4596c02 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTotalOrderPartitioner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTotalOrderPartitioner.java @@ -35,52 +35,52 @@ import org.apache.hadoop.mapreduce.MRJobConfig; public class TezTotalOrderPartitioner implements Partitioner, Configurable { - private static final Logger LOG = LoggerFactory.getLogger(TezTotalOrderPartitioner.class); + private static final Logger LOG = LoggerFactory.getLogger(TezTotalOrderPartitioner.class); - private Partitioner partitioner; + private Partitioner partitioner; - private static final String TEZ_RUNTIME_FRAMEWORK_PREFIX = "tez.runtime.framework."; - public static final String TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS = TEZ_RUNTIME_FRAMEWORK_PREFIX - + "num.expected.partitions"; + private static final String TEZ_RUNTIME_FRAMEWORK_PREFIX = "tez.runtime.framework."; + public static final String TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS = + TEZ_RUNTIME_FRAMEWORK_PREFIX + "num.expected.partitions"; - @Override - public void configure(JobConf job) { - if (partitioner == null) { - configurePartitioner(new JobConf(job)); - } - } + @Override + public void configure(JobConf job) { +if (partitioner == null) { + configurePartitioner(new JobConf(job)); +} + } - @Override - public void setConf(Configuration conf) { - // walk-around of TEZ-1403 - if (partitioner == null) { - configurePartitioner(new JobConf(conf)); - } - } + @Override + public void setConf(Configuration conf) { +// walk-around of TEZ-1403 +if (partitioner == null) { + configurePartitioner(new JobConf(conf)); +} + } - public int getPartition(HiveKey key, Object value, int numPartitions) { - return partitioner.getPartition(key, value, numPartitions); - } + public int getPartition(HiveKey key, Object value, int numPartitions) { +return partitioner.getPartition(key, value, numPartitions); + } - @Override - public Configuration getConf() { - return null; - } + @Override + public Configuration getConf() { +return null; + } - private void configurePartitioner(JobConf conf) { - LOG.info(TotalOrderPartitioner.getPartitionFile(conf)); - // make the HiveKey assumption - conf.setMapOutputKeyClass(HiveKey.class); - LOG.info(conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); - // remove the Tez fast serialization factory (TEZ-1288) - // this one skips the len prefix, so that the sorter can assume byte-order == - // sort-order - conf.setStrings(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, JavaSerialization.class.getName(), - WritableSerialization.class.getName()); - int partitions = conf.getInt(TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, -1); - // get the tez partitioning and feed it into the MR config - conf.setInt(MRJobConfig.NUM_REDUCES, partitions); - partitioner = new TotalOrderPartitioner(); - partitioner.configure(conf); - } + private void configurePartitioner(JobConf conf) { +LOG.info(TotalOrderPartitioner.getPartitionFile(conf)); +// make the HiveKey assumption +conf.setMapOutputKeyClass(HiveKey.class); +LOG.info(conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); +// remove the Tez fast serialization factory (TEZ-1288) +// this one skips the len prefix, so that the sorter can assume byte-order == +// sort-order +conf.setStrings(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, +JavaSerialization.class.getName(), WritableSerialization.class.getNa
[hive] branch master updated: HIVE-22948: QueryCache: Treat query cache locations as temporary storage (Gopal V, reviewed by Vineet Garg)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 5dda1b1 HIVE-22948: QueryCache: Treat query cache locations as temporary storage (Gopal V, reviewed by Vineet Garg) 5dda1b1 is described below commit 5dda1b1ae283493eea2387843cc35440363dfce3 Author: Gopal V AuthorDate: Sun Mar 29 23:26:18 2020 -0700 HIVE-22948: QueryCache: Treat query cache locations as temporary storage (Gopal V, reviewed by Vineet Garg) Signed-off-by: Gopal V --- .../plugin/TestHiveAuthorizerCheckInvocation.java| 14 ++ ql/src/java/org/apache/hadoop/hive/ql/Context.java | 16 .../apache/hadoop/hive/ql/parse/SemanticAnalyzer.java| 4 +++- 3 files changed, 33 insertions(+), 1 deletion(-) 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 09d142a..37bb6ad 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 @@ -98,6 +98,7 @@ public class TestHiveAuthorizerCheckInvocation { 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.setBoolVar(ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED, true); conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); SessionState.start(conf); @@ -161,6 +162,19 @@ public class TestHiveAuthorizerCheckInvocation { } @Test + public void testQueryCacheIgnored() throws Exception { + +reset(mockedAuthorizer); +int status = driver.compile("select i from " + acidTableName ++ " where i > 0 ", true); +assertEquals(0, status); +List outputs = getHivePrivilegeObjectInputs().getRight(); +List inputs = getHivePrivilegeObjectInputs().getLeft(); +assertEquals("No outputs for a select", 0, outputs.size()); +assertEquals("One input for this select", 1, inputs.size()); + } + + @Test public void testInputSomeColumnsUsedJoin() throws Exception { reset(mockedAuthorizer); 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 a85b94c..fd627c6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -727,6 +727,22 @@ public class Context { (uriStr.indexOf(MR_PREFIX) != -1); } + /** + * Check if the path is a result cache dir for this query. This doesn't work unless the result + * paths have already been set in SemanticAnalyzer::getDestinationFilePath to prevent someone from + * overriding LOCATION in a create table command to overwrite cached results + * + * @param destinationPath + * @return true if the path is a result cache dir + */ + + public boolean isResultCacheDir(Path destinationPath) { +if (this.fsResultCacheDirs != null) { + return this.fsResultCacheDirs.equals(destinationPath); +} +return false; + } + public Path getMRTmpPath(URI uri) { return new Path(getStagingDir(new Path(uri), !isExplainSkipExecution()), MR_PREFIX + nextPathId()); } 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 679ae2e..98317d1 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 @@ -7767,7 +7767,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { } boolean isDestTempFile = true; - if (!ctx.isMRTmpFileURI(destinationPath.toUri().toString())) { + if (ctx.isMRTmpFileURI(destinationPath.toUri().toString()) == false + && ctx.isResultCacheDir(destinationPath) == false) { +// not a temp dir and not a result cache dir idToTableNameMap.put(String.valueOf(destTableId), destinationPath.toUri().toString()); currentTableId = destTableId; destTableId++;
[hive] branch master updated: HIVE-22315: Support Decimal64 column division with decimal64 scalar (Ramesh Kumar Thangarajan, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 1c189b6 HIVE-22315: Support Decimal64 column division with decimal64 scalar (Ramesh Kumar Thangarajan, reviewed by Gopal V) 1c189b6 is described below commit 1c189b6da4b69647bda240c72aec490cb991 Author: Ramesh Kumar Thangarajan AuthorDate: Thu Oct 24 17:33:37 2019 -0700 HIVE-22315: Support Decimal64 column division with decimal64 scalar (Ramesh Kumar Thangarajan, reviewed by Gopal V) Signed-off-by: Gopal V --- data/files/decimal64table.csv | 1000 .../test/resources/testconfiguration.properties|1 + .../Decimal64ColumnDivideDecimal64Scalar.txt | 268 ++ .../hive/ql/exec/vector/VectorizationContext.java |8 + .../hive/ql/udf/generic/GenericUDFOPDivide.java|2 +- .../vector_decimal64_div_decimal64scalar.q |8 + .../vector_decimal64_div_decimal64scalar.q.out | 304 ++ .../llap/vector_decimal_expressions.q.out |8 +- .../clientpositive/llap/vector_decimal_udf.q.out | 12 +- .../vector_decimal64_div_decimal64scalar.q.out | 248 + .../vector_decimal_col_scalar_division.q.out |6 +- .../vector_decimal_expressions.q.out |6 +- .../apache/hadoop/hive/tools/GenVectorCode.java|4 + 13 files changed, 1858 insertions(+), 17 deletions(-) diff --git a/data/files/decimal64table.csv b/data/files/decimal64table.csv new file mode 100644 index 000..1038ddf --- /dev/null +++ b/data/files/decimal64table.csv @@ -0,0 +1,1000 @@ +1001,1,101,11 +1002,2,102,12 +1003,3,103,13 +1004,4,104,14 +1005,5,105,15 +1006,6,106,16 +1007,7,107,17 +1008,8,108,18 +1009,9,109,19 +1010,10,110,20 +1011,11,111,21 +1012,12,112,22 +1013,13,113,23 +1014,14,114,24 +1015,15,115,25 +1016,16,116,26 +1017,17,117,27 +1018,18,118,28 +1019,19,119,29 +1020,20,120,30 +1021,21,121,31 +1022,22,122,32 +1023,23,123,33 +1024,24,124,34 +1025,25,125,35 +1026,26,126,36 +1027,27,127,37 +1028,28,128,38 +1029,29,129,39 +1030,30,130,40 +1031,31,131,41 +1032,32,132,42 +1033,33,133,43 +1034,34,134,44 +1035,35,135,45 +1036,36,136,46 +1037,37,137,47 +1038,38,138,48 +1039,39,139,49 +1040,40,140,50 +1041,41,141,51 +1042,42,142,52 +1043,43,143,53 +1044,44,144,54 +1045,45,145,55 +1046,46,146,56 +1047,47,147,57 +1048,48,148,58 +1049,49,149,59 +1050,50,150,60 +1051,51,151,61 +1052,52,152,62 +1053,53,153,63 +1054,54,154,64 +1055,55,155,65 +1056,56,156,66 +1057,57,157,67 +1058,58,158,68 +1059,59,159,69 +1060,60,160,70 +1061,61,161,71 +1062,62,162,72 +1063,63,163,73 +1064,64,164,74 +1065,65,165,75 +1066,66,166,76 +1067,67,167,77 +1068,68,168,78 +1069,69,169,79 +1070,70,170,80 +1071,71,171,81 +1072,72,172,82 +1073,73,173,83 +1074,74,174,84 +1075,75,175,85 +1076,76,176,86 +1077,77,177,87 +1078,78,178,88 +1079,79,179,89 +1080,80,180,90 +1081,81,181,91 +1082,82,182,92 +1083,83,183,93 +1084,84,184,94 +1085,85,185,95 +1086,86,186,96 +1087,87,187,97 +1088,88,188,98 +1089,89,189,99 +1090,90,190,100 +1091,91,191,101 +1092,92,192,102 +1093,93,193,103 +1094,94,194,104 +1095,95,195,105 +1096,96,196,106 +1097,97,197,107 +1098,98,198,108 +1099,99,199,109 +1100,100,200,110 +1101,101,201,111 +1102,102,202,112 +1103,103,203,113 +1104,104,204,114 +1105,105,205,115 +1106,106,206,116 +1107,107,207,117 +1108,108,208,118 +1109,109,209,119 +1110,110,210,120 +,111,211,121 +1112,112,212,122 +1113,113,213,123 +1114,114,214,124 +1115,115,215,125 +1116,116,216,126 +1117,117,217,127 +1118,118,218,128 +1119,119,219,129 +1120,120,220,130 +1121,121,221,131 +1122,122,222,132 +1123,123,223,133 +1124,124,224,134 +1125,125,225,135 +1126,126,226,136 +1127,127,227,137 +1128,128,228,138 +1129,129,229,139 +1130,130,230,140 +1131,131,231,141 +1132,132,232,142 +1133,133,233,143 +1134,134,234,144 +1135,135,235,145 +1136,136,236,146 +1137,137,237,147 +1138,138,238,148 +1139,139,239,149 +1140,140,240,150 +1141,141,241,151 +1142,142,242,152 +1143,143,243,153 +1144,144,244,154 +1145,145,245,155 +1146,146,246,156 +1147,147,247,157 +1148,148,248,158 +1149,149,249,159 +1150,150,250,160 +1151,151,251,161 +1152,152,252,162 +1153,153,253,163 +1154,154,254,164 +1155,155,255,165 +1156,156,256,166 +1157,157,257,167 +1158,158,258,168 +1159,159,259,169 +1160,160,260,170 +1161,161,261,171 +1162,162,262,172 +1163,163,263,173 +1164,164,264,174 +1165,165,265,175 +1166,166,266,176 +1167,167,267,177 +1168,168,268,178 +1169,169,269,179 +1170,170,270,180 +1171,171,271,181 +1172,172,272,182 +1173,173,273,183 +1174,174,274,184 +1175,175,275,185 +1176,176,276,186 +1177,177,277,187 +1178,178,278,188 +1179,179,279,189 +1180,180,280,190 +1181,181,281,191 +1182,182,282,192 +1183,183,283,193 +1184,184,284,194 +1185,185,285,195 +1186,186,286,196 +1187,187,287,197 +1188,188,288,198 +1189,189,289,199 +1190,190,290,200
[hive] branch master updated: HIVE-22214: Explain vectorization should disable user level explain (Addendum)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new b9b1271 HIVE-22214: Explain vectorization should disable user level explain (Addendum) b9b1271 is described below commit b9b12715afef55bfa43992313652aeb03aaedf3f Author: Gopal V AuthorDate: Tue Sep 24 08:05:52 2019 -0700 HIVE-22214: Explain vectorization should disable user level explain (Addendum) --- .../test/results/clientpositive/tez/topnkey.q.out | 132 +++-- .../tez/vector_join_part_col_char.q.out| 139 +++-- .../clientpositive/tez/vector_topnkey.q.out| 619 + 3 files changed, 725 insertions(+), 165 deletions(-) diff --git a/ql/src/test/results/clientpositive/tez/topnkey.q.out b/ql/src/test/results/clientpositive/tez/topnkey.q.out index e786c39..3267f79 100644 --- a/ql/src/test/results/clientpositive/tez/topnkey.q.out +++ b/ql/src/test/results/clientpositive/tez/topnkey.q.out @@ -118,46 +118,102 @@ SELECT src1.key, src2.value FROM src src1 JOIN src src2 ON (src1.key = src2.key) POSTHOOK: type: QUERY POSTHOOK: Input: default@src POSTHOOK: Output: hdfs://### HDFS PATH ### -Plan optimized by CBO. +PLAN VECTORIZATION: + enabled: false + enabledConditionsNotMet: [hive.vectorized.execution.enabled IS false] -Vertex dependency in root stage -Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) -Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 -Stage-0 - Fetch Operator -limit:5 -Stage-1 - Reducer 3 - File Output Operator [FS_13] -Limit [LIM_12] (rows=5 width=178) - Number of rows:5 - Select Operator [SEL_11] (rows=791 width=178) -Output:["_col0","_col1"] - <-Reducer 2 [SIMPLE_EDGE] -SHUFFLE [RS_10] - Select Operator [SEL_9] (rows=791 width=178) -Output:["_col0","_col1"] -Merge Join Operator [MERGEJOIN_28] (rows=791 width=178) - Conds:RS_6._col0=RS_7._col0(Inner),Output:["_col0","_col2"] -<-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_6] -PartitionCols:_col0 -Select Operator [SEL_2] (rows=500 width=87) - Output:["_col0"] - Filter Operator [FIL_16] (rows=500 width=87) -predicate:key is not null -TableScan [TS_0] (rows=500 width=87) - default@src,src1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] -<-Map 4 [SIMPLE_EDGE] - SHUFFLE [RS_7] -PartitionCols:_col0 -Select Operator [SEL_5] (rows=500 width=178) - Output:["_col0","_col1"] - Filter Operator [FIL_17] (rows=500 width=178) -predicate:key is not null -TableScan [TS_3] (rows=500 width=178) - default@src,src2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] +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: src1 + filterExpr: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + 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) +sort order: + +Map-reduce partition columns: _col0 (type: string) +Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE +Map 4 +Map Operator Tree: +TableScan + alias: src2 + filterExpr: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE +
[hive] branch master updated: HIVE-22161: UDF: FunctionRegistry synchronizes on org.apache.hadoop.hive.ql.udf.UDFType class (Gopal V, reviewed by Ashutosh Chauhan)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new ebcc9bc HIVE-22161: UDF: FunctionRegistry synchronizes on org.apache.hadoop.hive.ql.udf.UDFType class (Gopal V, reviewed by Ashutosh Chauhan) ebcc9bc is described below commit ebcc9bc56ce92877f5e695b5c15654e380c46286 Author: Gopal V AuthorDate: Tue Sep 3 17:20:07 2019 -0700 HIVE-22161: UDF: FunctionRegistry synchronizes on org.apache.hadoop.hive.ql.udf.UDFType class (Gopal V, reviewed by Ashutosh Chauhan) Signed-off-by: Gopal V --- .../java/org/apache/hive/common/util/AnnotationUtils.java | 14 ++ 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/common/src/java/org/apache/hive/common/util/AnnotationUtils.java b/common/src/java/org/apache/hive/common/util/AnnotationUtils.java index a73faca..bfbaea6 100644 --- a/common/src/java/org/apache/hive/common/util/AnnotationUtils.java +++ b/common/src/java/org/apache/hive/common/util/AnnotationUtils.java @@ -23,17 +23,15 @@ import java.lang.reflect.Method; public class AnnotationUtils { - // to avoid https://bugs.openjdk.java.net/browse/JDK-7122142 + // until JDK8, this had a lock around annotationClass to avoid + // https://bugs.openjdk.java.net/browse/JDK-7122142 public static T getAnnotation(Class clazz, Class annotationClass) { -synchronized (annotationClass) { - return clazz.getAnnotation(annotationClass); -} +return clazz.getAnnotation(annotationClass); } - // to avoid https://bugs.openjdk.java.net/browse/JDK-7122142 + // until JDK8, this had a lock around annotationClass to avoid + // https://bugs.openjdk.java.net/browse/JDK-7122142 public static T getAnnotation(Method method, Class annotationClass) { -synchronized (annotationClass) { - return method.getAnnotation(annotationClass); -} +return method.getAnnotation(annotationClass); } }
[hive] branch master updated: HIVE-22075: Fix the max-reducers=1 regression from HIVE-14200 (Gopal V, reviewed by Jesus Camacho Rodriguez)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new dc24ce6 HIVE-22075: Fix the max-reducers=1 regression from HIVE-14200 (Gopal V, reviewed by Jesus Camacho Rodriguez) dc24ce6 is described below commit dc24ce69d06d3371cbec9125e43e122a6dbf47ca Author: Gopal V AuthorDate: Tue Aug 13 14:22:29 2019 -0700 HIVE-22075: Fix the max-reducers=1 regression from HIVE-14200 (Gopal V, reviewed by Jesus Camacho Rodriguez) Signed-off-by: Gopal V --- .../hadoop/hive/ql/optimizer/physical/LlapDecider.java | 17 ++--- 1 file changed, 10 insertions(+), 7 deletions(-) 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 0053997..2f2f04f 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,19 +173,22 @@ public class LlapDecider implements PhysicalPlanResolver { return; // Not based on ARP and cannot assume uniform distribution, bail. } clusterState.initClusterInfo(); - int targetCount = 0; + final int targetCount; + final int executorCount; + final int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); if (!clusterState.hasClusterInfo()) { LOG.warn("Cannot determine LLAP cluster information"); -targetCount = (int)Math.ceil(minReducersPerExec * 1 * executorsPerNode); +executorCount = executorsPerNode; // assume 1 node } else { -targetCount = (int)Math.ceil(minReducersPerExec * (clusterState.getKnownExecutorCount() -+ clusterState.getNodeCountWithUnknownExecutors() * executorsPerNode)); +executorCount = +clusterState.getKnownExecutorCount() + executorsPerNode +* clusterState.getNodeCountWithUnknownExecutors(); } - // We only increase the targets here. + targetCount = Math.min(maxReducers, (int) Math.ceil(minReducersPerExec * executorCount)); + // We only increase the targets here, but we stay below maxReducers if (reduceWork.isAutoReduceParallelism()) { // Do not exceed the configured max reducers. -int newMin = Math.min(conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS), -Math.max(reduceWork.getMinReduceTasks(), targetCount)); +int newMin = Math.min(maxReducers, Math.max(reduceWork.getMinReduceTasks(), targetCount)); if (newMin < reduceWork.getMaxReduceTasks()) { reduceWork.setMinReduceTasks(newMin); reduceWork.getEdgePropRef().setAutoReduce(conf, true, newMin,
[hive] branch master updated: HIVE-22008: LIKE Operator should match multi-line input (Gopal V, reviewed by Jesus Camacho Rodriguez)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new d7475aa HIVE-22008: LIKE Operator should match multi-line input (Gopal V, reviewed by Jesus Camacho Rodriguez) d7475aa is described below commit d7475aa98f6a2fc813e2e1c0ad99f902cb28cc00 Author: Gopal V AuthorDate: Thu Aug 1 23:24:25 2019 -0700 HIVE-22008: LIKE Operator should match multi-line input (Gopal V, reviewed by Jesus Camacho Rodriguez) Signed-off-by: Gopal V --- .../org/apache/hadoop/hive/ql/udf/UDFLike.java | 2 +- ql/src/test/queries/clientpositive/udf_like.q | 5 + ql/src/test/results/clientpositive/udf_like.q.out | 26 ++ 3 files changed, 32 insertions(+), 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLike.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLike.java index 793e8a7..4889b2e 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLike.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLike.java @@ -184,7 +184,7 @@ public class UDFLike extends UDF { parseSimplePattern(strLikePattern); if (type == PatternType.COMPLEX) { -p = Pattern.compile(likePatternToRegExp(strLikePattern)); +p = Pattern.compile(likePatternToRegExp(strLikePattern), Pattern.DOTALL); } } diff --git a/ql/src/test/queries/clientpositive/udf_like.q b/ql/src/test/queries/clientpositive/udf_like.q index d7728e7..a4a7251 100644 --- a/ql/src/test/queries/clientpositive/udf_like.q +++ b/ql/src/test/queries/clientpositive/udf_like.q @@ -23,3 +23,8 @@ SELECT '1+2' LIKE '_+_', '|||' LIKE '|_|', '+++' LIKE '1+_' FROM src tablesample (1 rows); + + +CREATE TEMPORARY TABLE SplitLines(`id` string) STORED AS ORC; +INSERT INTO SplitLines SELECT 'withdraw\ncash'; +SELECT `id` LIKE '%withdraw%cash' FROM SplitLines ; diff --git a/ql/src/test/results/clientpositive/udf_like.q.out b/ql/src/test/results/clientpositive/udf_like.q.out index 3a131ac..8a0bb98 100644 --- a/ql/src/test/results/clientpositive/udf_like.q.out +++ b/ql/src/test/results/clientpositive/udf_like.q.out @@ -84,3 +84,29 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@src A masked pattern was here true truefalse truefalse +PREHOOK: query: CREATE TEMPORARY TABLE SplitLines(`id` string) STORED AS ORC +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@SplitLines +POSTHOOK: query: CREATE TEMPORARY TABLE SplitLines(`id` string) STORED AS ORC +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@SplitLines +PREHOOK: query: INSERT INTO SplitLines SELECT 'withdraw\ncash' +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@splitlines +POSTHOOK: query: INSERT INTO SplitLines SELECT 'withdraw\ncash' +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@splitlines +POSTHOOK: Lineage: splitlines.id SIMPLE [] +PREHOOK: query: SELECT `id` LIKE '%withdraw%cash' FROM SplitLines +PREHOOK: type: QUERY +PREHOOK: Input: default@splitlines + A masked pattern was here +POSTHOOK: query: SELECT `id` LIKE '%withdraw%cash' FROM SplitLines +POSTHOOK: type: QUERY +POSTHOOK: Input: default@splitlines + A masked pattern was here +true
[hive] branch master updated: HIVE-15177: Authentication with hive fails when kerberos auth type is set to fromSubject and principal contains _HOST (Oliver Draese, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 57c4217 HIVE-15177: Authentication with hive fails when kerberos auth type is set to fromSubject and principal contains _HOST (Oliver Draese, reviewed by Gopal V) 57c4217 is described below commit 57c4217475856271233d66a7639fb70288d47a43 Author: Oliver Draese AuthorDate: Thu Jun 27 16:20:52 2019 -0700 HIVE-15177: Authentication with hive fails when kerberos auth type is set to fromSubject and principal contains _HOST (Oliver Draese, reviewed by Gopal V) Signed-off-by: Gopal V --- .gitignore | 3 +++ .../hive/service/auth/KerberosSaslHelper.java | 21 ++--- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/.gitignore b/.gitignore index 750f17b..442c132 100644 --- a/.gitignore +++ b/.gitignore @@ -30,7 +30,10 @@ hcatalog/webhcat/svr/target conf/hive-default.xml.template itests/hive-blobstore/src/test/resources/blobstore-conf.xml .DS_Store +.factorypath patchprocess standalone-metastore/src/gen/version standalone-metastore/metastore-common/src/gen/version standalone-metastore/metastore-server/src/gen/version +launch.json +settings.json diff --git a/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java index 8b2be9d..14d2008 100644 --- a/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ b/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -24,6 +24,7 @@ import javax.security.sasl.SaslException; import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.Server; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hive.service.cli.thrift.ThriftCLIService; import org.apache.hive.service.rpc.thrift.TCLIService; import org.apache.hive.service.rpc.thrift.TCLIService.Iface; @@ -49,7 +50,7 @@ public final class KerberosSaslHelper { } if (assumeSubject) { -return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); +return createSubjectAssumedTransport(principal, host, underlyingTransport, saslProps); } else { HadoopThriftAuthBridge.Client authBridge = HadoopThriftAuthBridge.getBridge().createClientWithConf("kerberos"); @@ -61,9 +62,23 @@ public final class KerberosSaslHelper { } } - public static TTransport createSubjectAssumedTransport(String principal, + /** + * Helper to wrap the {@code underlyingTransport} into an assumed kerberos principal. + * The function is used for kerberos based authentication, where {@code kerberosAuthType} + * is set to {@code fromSubject}. If also performs a substitution of {@code _HOST} to the + * local host name, if required. + * + * @param principal The kerberos principal to assume + * @param host Host, used to replace the {@code _HOST} with + * @param underlyingTransport The I/O transport to wrap + * @param saslProps SASL property map + * @return The wrapped transport + * @throws IOException + */ + public static TTransport createSubjectAssumedTransport(String principal, String host, TTransport underlyingTransport, Map saslProps) throws IOException { -String[] names = principal.split("[/@]"); +String resolvedPrincipal = SecurityUtil.getServerPrincipal(principal, host); +String[] names = resolvedPrincipal.split("[/@]"); try { TTransport saslTransport = new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null,
[hive] branch master updated: HIVE-21815: Stats in ORC file are parsed twice (Krisztian Kasa, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new acaa98c HIVE-21815: Stats in ORC file are parsed twice (Krisztian Kasa, reviewed by Gopal V) acaa98c is described below commit acaa98c5e40518b76a71c85aea0a84d1295b5ef2 Author: Krisztian Kasa AuthorDate: Wed Jun 12 19:19:13 2019 -0700 HIVE-21815: Stats in ORC file are parsed twice (Krisztian Kasa, reviewed by Gopal V) Signed-off-by: Gopal V --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java | 7 +-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 3878bba..707e38c 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 @@ -1728,9 +1728,12 @@ public class OrcInputFormat implements InputFormat, if (context.cacheStripeDetails) { context.footerCache.put(new FooterCacheKey(fsFileId, file.getPath()), orcTail); } +stripes = orcReader.getStripes(); +stripeStats = orcReader.getStripeStatistics(); + } else { +stripes = orcTail.getStripes(); +stripeStats = orcTail.getStripeStatistics(); } - stripes = orcTail.getStripes(); - stripeStats = orcTail.getStripeStatistics(); fileTypes = orcTail.getTypes(); TypeDescription fileSchema = OrcUtils.convertTypeFromProtobuf(fileTypes, 0); Reader.Options readerOptions = new Reader.Options(context.conf);
[hive] branch master updated: HIVE-21768: JDBC: Strip the default union prefix for un-enclosed UNION queries (Gopal V, reviewed by Jesus Camacho Rodriguez)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new a26dc36 HIVE-21768: JDBC: Strip the default union prefix for un-enclosed UNION queries (Gopal V, reviewed by Jesus Camacho Rodriguez) a26dc36 is described below commit a26dc36c04fc3a92ce98b472d9e9ed46e5de7b0e Author: Gopal V AuthorDate: Tue May 28 15:07:12 2019 -0700 HIVE-21768: JDBC: Strip the default union prefix for un-enclosed UNION queries (Gopal V, reviewed by Jesus Camacho Rodriguez) --- .../java/org/apache/hive/jdbc/TestJdbcDriver2.java | 12 .../java/org/apache/hadoop/hive/ql/parse/HiveParser.g | 2 +- .../apache/hadoop/hive/ql/parse/SemanticAnalyzer.java | 3 ++- ql/src/test/results/clientpositive/input_part7.q.out | 4 ++-- .../clientpositive/spark/optimize_nullscan.q.out | 18 +- .../test/results/clientpositive/spark/union_ppr.q.out | 8 ql/src/test/results/clientpositive/union_ppr.q.out | 4 ++-- 7 files changed, 32 insertions(+), 19 deletions(-) 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 654bdf8..79e4818 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 @@ -3166,6 +3166,18 @@ public class TestJdbcDriver2 { } @Test + public void testUnionUniqueColumnNames() throws Exception { +HiveStatement stmt = (HiveStatement) con.createStatement(); + +stmt.execute("SET hive.resultset.use.unique.column.names=true"); +ResultSet rs = stmt.executeQuery("select 1 UNION ALL select 2"); +ResultSetMetaData metaData = rs.getMetaData(); +assertEquals("_c0", metaData.getColumnLabel(1)); +assertTrue("There's no . for the UNION column name", !metaData.getColumnLabel(1).contains(".")); +stmt.close(); + } + + @Test public void testGetQueryId() throws Exception { HiveStatement stmt = (HiveStatement) con.createStatement(); HiveStatement stmt1 = (HiveStatement) con.createStatement(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g index 46fa563..bf9aa39 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g @@ -747,7 +747,7 @@ import org.apache.hadoop.hive.conf.HiveConf; // counter to generate unique union aliases private int aliasCounter; private String generateUnionAlias() { -return "_u" + (++aliasCounter); +return "__u" + (++aliasCounter); } private char [] excludedCharForColumnName = {'.', ':'}; private boolean containExcludedCharForCreateTableColumnName(String input) { 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 aadbf38..edb20ae 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 @@ -12825,7 +12825,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { } qualifiedColName = rr.reverseLookup(colInfo.getInternalName()); - if (useTabAliasIfAvailable && qualifiedColName[0] != null && !qualifiedColName[0].isEmpty()) { + // __u is a UNION ALL placeholder name + if (useTabAliasIfAvailable && qualifiedColName[0] != null && (!qualifiedColName[0].isEmpty()) && (!qualifiedColName[0].startsWith("__u"))) { colName = qualifiedColName[0] + "." + qualifiedColName[1]; } else { colName = qualifiedColName[1]; diff --git a/ql/src/test/results/clientpositive/input_part7.q.out b/ql/src/test/results/clientpositive/input_part7.q.out index d1838e7..15a388e 100644 --- a/ql/src/test/results/clientpositive/input_part7.q.out +++ b/ql/src/test/results/clientpositive/input_part7.q.out @@ -184,8 +184,8 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: -/srcpart/ds=2008-04-08/hr=11 [a:_u1-subquery1:x, a:_u1-subquery2:y] -/srcpart/ds=2008-04-08/hr=12 [a:_u1-subquery1:x, a:_u1-subquery2:y] +/srcpart/ds=2008-04-08/hr=11 [a:__u1-subquery1:x, a:__u1-subquery2:y] +/srcpart/ds=2008-04-08/hr=12 [a:__u1-subquery1:x, a:__u1-subquery2:y] Needs Tagging: false Reduce Operator Tree: Select Operator diff --git a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/spark/optimiz
[hive] branch master updated: HIVE-21646: Tez: Prevent TezTasks from escaping thread logging context (Gopal V, reviewed by Ashutosh Chauhan)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 143ea96 HIVE-21646: Tez: Prevent TezTasks from escaping thread logging context (Gopal V, reviewed by Ashutosh Chauhan) 143ea96 is described below commit 143ea9663249a41de5aae3dd0987df8fe400a906 Author: Gopal V AuthorDate: Tue May 21 22:54:33 2019 -0700 HIVE-21646: Tez: Prevent TezTasks from escaping thread logging context (Gopal V, reviewed by Ashutosh Chauhan) --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java | 5 + 1 file changed, 5 insertions(+) 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 f2ed07a..dd7ccd4 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 @@ -631,6 +631,11 @@ public class TezTask extends Task { } @Override + public boolean canExecuteInParallel() { +return false; + } + + @Override public Collection getMapWork() { List result = new LinkedList(); TezWork work = getWork();
[hive] branch master updated: HIVE-21641: Llap external client returns decimal columns in different precision/scale as compared to beeline (Shubham Chaurasia, via Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new ff98efa HIVE-21641: Llap external client returns decimal columns in different precision/scale as compared to beeline (Shubham Chaurasia, via Gopal V) ff98efa is described below commit ff98efa7c6f2b241d8fddd0ac8dc55e817ecb234 Author: Shubham Chaurasia AuthorDate: Mon May 6 22:06:19 2019 -0700 HIVE-21641: Llap external client returns decimal columns in different precision/scale as compared to beeline (Shubham Chaurasia, via Gopal V) Signed-off-by: Gopal V --- .../hive/jdbc/TestJdbcGenericUDTFGetSplits.java| 49 +++--- .../apache/hadoop/hive/ql/parse/ParseUtils.java| 2 +- 2 files changed, 44 insertions(+), 7 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java index b94868b..f6f64b8 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java @@ -34,18 +34,17 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.TimeUnit; 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.llap.FieldDesc; import org.apache.hadoop.hive.llap.LlapBaseInputFormat; -import org.apache.hadoop.hive.metastore.api.WMTrigger; -import org.apache.hadoop.hive.ql.wm.Action; -import org.apache.hadoop.hive.ql.wm.ExecutionTrigger; -import org.apache.hadoop.hive.ql.wm.Expression; -import org.apache.hadoop.hive.ql.wm.ExpressionFactory; -import org.apache.hadoop.hive.ql.wm.Trigger; +import org.apache.hadoop.hive.llap.LlapInputSplit; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.mapred.JobConf; import org.apache.hive.jdbc.miniHS2.MiniHS2; import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType; import org.junit.After; @@ -138,6 +137,44 @@ public class TestJdbcGenericUDTFGetSplits { runQuery(query, setCmds, 10); } + @Test + public void testDecimalPrecisionAndScale() throws Exception { +try (Statement stmt = hs2Conn.createStatement()) { + stmt.execute("CREATE TABLE decimal_test_table(decimal_col DECIMAL(6,2))"); + stmt.execute("INSERT INTO decimal_test_table VALUES(2507.92)"); + + ResultSet rs = stmt.executeQuery("SELECT * FROM decimal_test_table"); + assertTrue(rs.next()); + rs.close(); + + String url = miniHS2.getJdbcURL(); + String user = System.getProperty("user.name"); + String pwd = user; + String handleId = UUID.randomUUID().toString(); + String sql = "SELECT avg(decimal_col)/3 FROM decimal_test_table"; + + // make request through llap-ext-client + JobConf job = new JobConf(conf); + job.set(LlapBaseInputFormat.URL_KEY, url); + job.set(LlapBaseInputFormat.USER_KEY, user); + job.set(LlapBaseInputFormat.PWD_KEY, pwd); + job.set(LlapBaseInputFormat.QUERY_KEY, sql); + job.set(LlapBaseInputFormat.HANDLE_ID, handleId); + + LlapBaseInputFormat llapBaseInputFormat = new LlapBaseInputFormat(); + //schema split + LlapInputSplit schemaSplit = (LlapInputSplit) llapBaseInputFormat.getSplits(job, 0)[0]; + assertNotNull(schemaSplit); + FieldDesc fieldDesc = schemaSplit.getSchema().getColumns().get(0); + DecimalTypeInfo type = (DecimalTypeInfo) fieldDesc.getTypeInfo(); + assertEquals(38, type.getPrecision()); + assertEquals(24, type.scale()); + + LlapBaseInputFormat.close(handleId); +} + } + + private void runQuery(final String query, final List setCmds, final int numRows) throws Exception { 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 07c65af..537af20 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 @@ -558,7 +558,7 @@ public final class ParseUtils { ctx.setIsLoadingMaterializedView(true); final ASTNode ast = parse(viewQuery, ctx); final CalcitePlanner analyzer = getAnalyzer(conf, ctx); -analyzer.genLogicalPlan(ast); +analyzer.analyze(ast, ctx); return analyzer.getResultSchema(); }
[hive] branch master updated: HIVE-21400: Vectorization: LazyBinarySerializeWrite allocates Field() within the loop (Gopal V, reviewed by Prasanth Jayachandran)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 5386a61 HIVE-21400: Vectorization: LazyBinarySerializeWrite allocates Field() within the loop (Gopal V, reviewed by Prasanth Jayachandran) 5386a61 is described below commit 5386a61372c15b3fae375c84462da3df7766a6ac Author: Gopal V AuthorDate: Wed Apr 24 15:37:15 2019 -0700 HIVE-21400: Vectorization: LazyBinarySerializeWrite allocates Field() within the loop (Gopal V, reviewed by Prasanth Jayachandran) --- .../lazybinary/fast/LazyBinarySerializeWrite.java | 17 ++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java index ec56b82..ae9111f 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java @@ -73,12 +73,13 @@ public class LazyBinarySerializeWrite implements SerializeWrite { private long[] scratchLongs; private byte[] scratchBuffer; - private Field root; + private final Field root; private Deque stack = new ArrayDeque<>(); private LazyBinarySerDe.BooleanRef warnedOnceNullMapKey; private static class Field { -Category type; +// Make sure the root.type never changes from STRUCT +final Category type; int fieldCount; int fieldIndex; @@ -90,6 +91,15 @@ public class LazyBinarySerializeWrite implements SerializeWrite { Field(Category type) { this.type = type; } + +public void clean() { + fieldCount = 0; + fieldIndex = 0; + byteSizeStart = 0; + start = 0; + nullOffset = 0; + nullByte = 0; +} } public LazyBinarySerializeWrite(int fieldCount) { @@ -101,6 +111,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite { // Not public since we must have the field count and other information. private LazyBinarySerializeWrite() { +this.root = new Field(STRUCT); } /* @@ -133,7 +144,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite { } private void resetWithoutOutput() { -root = new Field(STRUCT); +root.clean(); root.fieldCount = rootFieldCount; stack.clear(); stack.push(root);
[hive] branch master updated: HIVE-21531: Vectorization: all NULL hashcodes are not computed using Murmur3 (Gopal V, reviewed by Ashutosh Chauhan)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new d7fbd63 HIVE-21531: Vectorization: all NULL hashcodes are not computed using Murmur3 (Gopal V, reviewed by Ashutosh Chauhan) d7fbd63 is described below commit d7fbd639ff0188b0e5d22ada50795b1a0784a0be Author: Gopal V AuthorDate: Wed Apr 24 15:34:32 2019 -0700 HIVE-21531: Vectorization: all NULL hashcodes are not computed using Murmur3 (Gopal V, reviewed by Ashutosh Chauhan) --- .../org/apache/hive/common/util/HashCodeUtil.java | 2 + .../test/resources/testconfiguration.properties| 1 + .../keyseries/VectorKeySeriesSerializedImpl.java | 5 +- .../VectorReduceSinkUniformHashOperator.java | 4 +- ql/src/test/queries/clientpositive/cbo_rp_limit.q | 4 +- .../queries/clientpositive/vector_union_null.q | 29 +++ .../results/clientpositive/llap/cbo_rp_limit.q.out | 20 +- .../clientpositive/llap/vector_union_null.q.out| 254 + .../results/clientpositive/vector_union_null.q.out | 173 ++ 9 files changed, 483 insertions(+), 9 deletions(-) diff --git a/common/src/java/org/apache/hive/common/util/HashCodeUtil.java b/common/src/java/org/apache/hive/common/util/HashCodeUtil.java index 1330cbe..d763b13 100644 --- a/common/src/java/org/apache/hive/common/util/HashCodeUtil.java +++ b/common/src/java/org/apache/hive/common/util/HashCodeUtil.java @@ -47,7 +47,9 @@ public class HashCodeUtil { } } + @Deprecated public static int calculateBytesHashCode(byte[] keyBytes, int keyStart, int keyLength) { +// Don't use this for ReduceSinkOperators return murmurHash(keyBytes, keyStart, keyLength); } diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 809c121..561d1f2 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -329,6 +329,7 @@ minillaplocal.shared.query.files=alter_merge_2_orc.q,\ vector_varchar_mapjoin1.q,\ vector_varchar_simple.q,\ vector_when_case_null.q,\ + vector_union_null.q,\ vectorization_0.q,\ vectorization_1.q,\ vectorization_10.q,\ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesSerializedImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesSerializedImpl.java index 77c9ecc..f81c636 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesSerializedImpl.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/keyseries/VectorKeySeriesSerializedImpl.java @@ -93,9 +93,8 @@ public abstract class VectorKeySeriesSerializedImpl * Batch compute the hash codes for all the serialized keys. * * NOTE: MAJOR MAJOR ASSUMPTION: - * We assume that HashCodeUtil.murmurHash produces the same result - * as MurmurHash.hash with seed = 0 (the method used by ReduceSinkOperator for - * UNIFORM distribution). + * We use Murmur3.hash32(seed=0) across the board for the ReduceSink UNIFORM distribution. + * Previous use of HashCodeUtil is deprecated. */ protected void computeSerializedHashCodes() { int offset = 0; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java index 324aa97..1201a1d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkUniformHashOperator.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.VectorDesc; import org.apache.hadoop.hive.serde2.ByteStream.Output; -import org.apache.hive.common.util.HashCodeUtil; +import org.apache.hive.common.util.Murmur3; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,7 +86,7 @@ public abstract class VectorReduceSinkUniformHashOperator extends VectorReduceSi int nullBytesLength = nullKeyOutput.getLength(); nullBytes = new byte[nullBytesLength]; System.arraycopy(nullKeyOutput.getData(), 0, nullBytes, 0, nullBytesLength); - nullKeyHashCode = HashCodeUtil.calculateBytesHashCode(nullBytes, 0, nullBytesLength); + nullKeyHashCode = Murmur3.hash32(nullBytes, 0, nullBytesLength, 0); } catch (Exception e) { throw new HiveException(e); } diff --git a/ql/src/test/queries/clientpositive/cbo_rp_limit.q b/ql/src/test/queries/clientpositive/cbo_rp_limit.q index 24c1eae..a70755a 100644 --- a/ql/src/test/qu
[hive] branch master updated (447f03f -> c279634)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 447f03f HIVE-21474: Preparation for bumping guava version (Peter Vary, reviewed by Slim Bouguerra and Zoltan Haindrich) new 741630c HIVE-21422: Add metrics to LRFU cache policy (Oliver Draese, reviewed by Gopal V) new c279634 HIVE-21183: Interrupt wait time for FileCacheCleanupThread (Oliver Draese, reviewed by Gopal V) The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../hive/llap/cache/FileCacheCleanupThread.java| 4 +- .../hive/llap/cache/LowLevelLrfuCachePolicy.java | 287 ++--- .../hive/llap/cache/SerDeLowLevelCacheImpl.java| 5 + 3 files changed, 259 insertions(+), 37 deletions(-)
[hive] 01/02: HIVE-21422: Add metrics to LRFU cache policy (Oliver Draese, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git commit 741630c1b81cb6928b935e3ca8c0ed46838b Author: Olli Draese AuthorDate: Mon Mar 25 09:16:37 2019 -0700 HIVE-21422: Add metrics to LRFU cache policy (Oliver Draese, reviewed by Gopal V) Signed-off-by: Gopal V --- .../hive/llap/cache/LowLevelLrfuCachePolicy.java | 287 ++--- 1 file changed, 251 insertions(+), 36 deletions(-) diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java index 704f2f1..759819d 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java @@ -27,6 +27,15 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.llap.LlapUtil; import org.apache.hadoop.hive.llap.cache.LowLevelCache.Priority; import org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl; +import org.apache.hadoop.hive.llap.io.metadata.MetadataCache.LlapMetadataBuffer; +import org.apache.hadoop.hive.llap.metrics.LlapMetricsSystem; +import org.apache.hadoop.hive.llap.metrics.MetricsUtils; +import org.apache.hadoop.metrics2.MetricsCollector; +import org.apache.hadoop.metrics2.MetricsInfo; +import org.apache.hadoop.metrics2.MetricsRecordBuilder; +import org.apache.hadoop.metrics2.MetricsSource; +import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.impl.MsInfo; /** * Implementation of the algorithm from "On the Existence of a Spectrum of Policies @@ -35,14 +44,14 @@ import org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl; */ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy { private final double lambda; - private final double f(long x) { + private double f(long x) { return Math.pow(0.5, lambda * x); } private static final double F0 = 1; // f(0) is always 1 - private final double touchPriority(long time, long lastAccess, double previous) { + private double touchPriority(long time, long lastAccess, double previous) { return F0 + f(time - lastAccess) * previous; } - private final double expirePriority(long time, long lastAccess, double previous) { + private double expirePriority(long time, long lastAccess, double previous) { return f(time - lastAccess) * previous; } @@ -65,6 +74,7 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy { private int heapSize = 0; private final int maxHeapSize; private EvictionListener evictionListener; + private final PolicyMetrics metrics; public LowLevelLrfuCachePolicy(int minBufferSize, long maxSize, Configuration conf) { lambda = HiveConf.getFloatVar(conf, HiveConf.ConfVars.LLAP_LRFU_LAMBDA); @@ -79,7 +89,18 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy { minBufferSize, lambda, maxHeapSize); heap = new LlapCacheableBuffer[maxHeapSize]; -listHead = listTail = null; +listHead = null; +listTail = null; + +String sessID = conf.get("llap.daemon.metrics.sessionid"); +if (null == sessID) { + sessID = ""; +} + +// register new metrics provider for this cache policy +metrics = new PolicyMetrics(sessID); +LlapMetricsSystem.instance().register("LowLevelLrfuCachePolicy-" + MetricsUtils.getHostName(), + null, metrics); } @Override @@ -110,8 +131,10 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy { // a locked item in either, it will remove it from cache; when we unlock, we are going to // put it back or update it, depending on whether this has happened. This should cause // most of the expensive cache update work to happen in unlock, not blocking processing. -if (buffer.indexInHeap != LlapCacheableBuffer.IN_LIST) return; -if (!listLock.tryLock()) return; +if (buffer.indexInHeap != LlapCacheableBuffer.IN_LIST || !listLock.tryLock()) { + return; +} + removeFromListAndUnlock(buffer); } @@ -149,7 +172,8 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy { listHead.prev = demoted; listHead = demoted; } else { -listHead = listTail = demoted; +listHead = demoted; +listTail = demoted; demoted.next = null; } } finally { @@ -179,7 +203,8 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy { LlapCacheableBuffer oldTail = null; listLock.lock(); try { - LlapCacheableBuffer current = oldTail = listTail; + LlapCacheableBuffer current = listTail; + oldTail = listTail; while (
[hive] 02/02: HIVE-21183: Interrupt wait time for FileCacheCleanupThread (Oliver Draese, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git commit c2796347fa5fcf524ba751e0b5b5aa19c65fd3c4 Author: Olli Draese AuthorDate: Mon Mar 25 09:17:08 2019 -0700 HIVE-21183: Interrupt wait time for FileCacheCleanupThread (Oliver Draese, reviewed by Gopal V) Signed-off-by: Gopal V --- .../org/apache/hadoop/hive/llap/cache/FileCacheCleanupThread.java| 4 +++- .../org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java| 5 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/FileCacheCleanupThread.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/FileCacheCleanupThread.java index 1835dad..af04f3b 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/FileCacheCleanupThread.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/FileCacheCleanupThread.java @@ -38,7 +38,7 @@ abstract class FileCacheCleanupThread extends Thread { this.newEvictions = newEvictions; this.approxCleanupIntervalSec = cleanupInterval; setDaemon(true); -setPriority(1); +setPriority(Thread.MIN_PRIORITY); } @Override @@ -61,6 +61,8 @@ abstract class FileCacheCleanupThread extends Thread { while (true) { int evictionsSinceLast = newEvictions.getAndSet(0); if (evictionsSinceLast > 0) break; + + // will be notified by SerDeLowLevelCacheImpl or timeout eventually synchronized (newEvictions) { newEvictions.wait(1); } diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java index a8f89ef..c41b34a 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SerDeLowLevelCacheImpl.java @@ -661,6 +661,11 @@ public class SerDeLowLevelCacheImpl implements BufferUsageManager, LlapIoDebugDu public final void notifyEvicted(MemoryBuffer buffer) { newEvictions.incrementAndGet(); + +// FileCacheCleanupThread might we waiting for eviction increment +synchronized(newEvictions) { + newEvictions.notifyAll(); +} } private final class CleanupThread extends FileCacheCleanupThread {
[hive] branch branch-3 updated: HIVE-21460: ACID: Load data followed by a select * query results in incorrect results (Vaibhav Gumashta, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch branch-3 in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/branch-3 by this push: new e9d1e03 HIVE-21460: ACID: Load data followed by a select * query results in incorrect results (Vaibhav Gumashta, reviewed by Gopal V) e9d1e03 is described below commit e9d1e03eeceaa44581d146026b1c4488df2acafb Author: Gopal V AuthorDate: Wed Mar 20 15:57:23 2019 -0700 HIVE-21460: ACID: Load data followed by a select * query results in incorrect results (Vaibhav Gumashta, reviewed by Gopal V) Signed-off-by: Gopal V (cherry picked from commit 25b14bec658b26ef36220e8697759190dd0f1ace) --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java index 929ea9b..4ccc028 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java @@ -724,10 +724,10 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader{ } } if (rowOffset > 0) { - minKey = new RecordIdentifier(0, bucketProperty, rowOffset - 1); + minKey = new RecordIdentifier(tfp.syntheticWriteId, bucketProperty, rowOffset - 1); } if (!isTail) { - maxKey = new RecordIdentifier(0, bucketProperty, rowOffset + rowLength - 1); + maxKey = new RecordIdentifier(tfp.syntheticWriteId, bucketProperty, rowOffset + rowLength - 1); } return new KeyInterval(minKey, maxKey); }
[hive] branch master updated: HIVE-21460: ACID: Load data followed by a select * query results in incorrect results (Vaibhav Gumashta, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 25b14be HIVE-21460: ACID: Load data followed by a select * query results in incorrect results (Vaibhav Gumashta, reviewed by Gopal V) 25b14be is described below commit 25b14bec658b26ef36220e8697759190dd0f1ace Author: Gopal V AuthorDate: Wed Mar 20 15:57:23 2019 -0700 HIVE-21460: ACID: Load data followed by a select * query results in incorrect results (Vaibhav Gumashta, reviewed by Gopal V) Signed-off-by: Gopal V --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java index 62a1061..fc7e7e1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java @@ -741,10 +741,10 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader{ } } if (rowOffset > 0) { - minKey = new RecordIdentifier(0, bucketProperty, rowOffset - 1); + minKey = new RecordIdentifier(tfp.syntheticWriteId, bucketProperty, rowOffset - 1); } if (!isTail) { - maxKey = new RecordIdentifier(0, bucketProperty, rowOffset + rowLength - 1); + maxKey = new RecordIdentifier(tfp.syntheticWriteId, bucketProperty, rowOffset + rowLength - 1); } return new KeyInterval(minKey, maxKey); }
[hive] 01/02: HIVE-21264: Improvements Around CharTypeInfo (David Mollitor, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git commit bfc44ff83b38d3ad4ce3cfadcc69ebbaff23e974 Author: David Mollitor AuthorDate: Sun Mar 10 22:45:56 2019 -0700 HIVE-21264: Improvements Around CharTypeInfo (David Mollitor, reviewed by Gopal V) Signed-off-by: Gopal V --- .../hive/serde2/typeinfo/BaseCharTypeInfo.java | 39 ++--- .../hadoop/hive/serde2/typeinfo/BaseCharUtils.java | 2 - .../hadoop/hive/serde2/typeinfo/CharTypeInfo.java | 22 -- .../hive/serde2/typeinfo/PrimitiveTypeInfo.java| 41 ++ .../hive/serde2/typeinfo/TypeInfoFactory.java | 6 +-- .../hive/serde2/typeinfo/VarcharTypeInfo.java | 22 -- .../hive/serde2/typeinfo/TestBaseCharTypeInfo.java | 50 ++ 7 files changed, 109 insertions(+), 73 deletions(-) diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java index 820fb4e..18de5a0 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharTypeInfo.java @@ -54,12 +54,16 @@ public abstract class BaseCharTypeInfo extends PrimitiveTypeInfo { return getQualifiedName(typeName, length); } - public static String getQualifiedName(String typeName, int length) { -StringBuilder sb = new StringBuilder(typeName); -sb.append("("); -sb.append(length); -sb.append(")"); -return sb.toString(); + /** + * Utility method to build the fully qualified data type. For example: + * (char,16) becomes char(16). + * + * @param typeName The name of the data type (char or varchar) + * @param length The maximum length of the data type + * @return A fully qualified field name + */ + protected String getQualifiedName(String typeName, int length) { +return typeName + '(' + length + ')'; } @Override @@ -67,4 +71,27 @@ public abstract class BaseCharTypeInfo extends PrimitiveTypeInfo { // type name should already be set by subclass return; } + + @Override + public int hashCode() { +final int prime = 31; +int result = super.hashCode(); +result = prime * result + length; +return result; + } + + @Override + public boolean equals(Object obj) { +if (this == obj) { + return true; +} +if (!super.equals(obj)) { + return false; +} +BaseCharTypeInfo other = (BaseCharTypeInfo) obj; +if (length != other.length) { + return false; +} +return true; + } } diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharUtils.java index 259e642..d3c53ff 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharUtils.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/BaseCharUtils.java @@ -22,8 +22,6 @@ import org.apache.hadoop.hive.common.type.HiveBaseChar; import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveVarchar; import org.apache.hadoop.hive.serde2.io.HiveBaseCharWritable; -import org.apache.hadoop.hive.serde2.io.HiveCharWritable; -import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; public class BaseCharUtils { diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/CharTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/CharTypeInfo.java index ef9b945..9b91317 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/CharTypeInfo.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/CharTypeInfo.java @@ -39,28 +39,6 @@ public class CharTypeInfo extends BaseCharTypeInfo { } @Override - public boolean equals(Object other) { -if (this == other) { - return true; -} -if (other == null || getClass() != other.getClass()) { - return false; -} - -CharTypeInfo pti = (CharTypeInfo) other; - -return this.typeName.equals(pti.typeName) && this.getLength() == pti.getLength(); - } - - /** - * Generate the hashCode for this TypeInfo. - */ - @Override - public int hashCode() { -return getQualifiedName().hashCode(); - } - - @Override public String toString() { return getQualifiedName(); } diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java index 97af49a..0394ffa 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/PrimitiveTypeInfo.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.serde2.typeinfo; import java.io.Se
[hive] branch master updated (13ec7c3 -> c3939da)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/hive.git. from 13ec7c3 HIVE-21390 : BI split strategy does not work for blob stores (Prasanth Jayachandran reviewed by Gopal V) new bfc44ff HIVE-21264: Improvements Around CharTypeInfo (David Mollitor, reviewed by Gopal V) new c3939da HIVE-21371: Make NonSyncByteArrayOutputStream Overflow Conscious (David Mollitor, reviewed by Gopal V) The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../common/io/NonSyncByteArrayOutputStream.java| 37 +-- .../hive/serde2/typeinfo/BaseCharTypeInfo.java | 39 .../hadoop/hive/serde2/typeinfo/BaseCharUtils.java | 2 -- .../hadoop/hive/serde2/typeinfo/CharTypeInfo.java | 22 .../hive/serde2/typeinfo/PrimitiveTypeInfo.java| 41 +- .../hive/serde2/typeinfo/TypeInfoFactory.java | 6 ++-- .../hive/serde2/typeinfo/VarcharTypeInfo.java | 22 .../hive/serde2/typeinfo/TestBaseCharTypeInfo.java | 37 +++ 8 files changed, 107 insertions(+), 99 deletions(-) copy common/src/test/org/apache/hive/common/util/TestShutdownHookManager.java => serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestBaseCharTypeInfo.java (54%)
[hive] 02/02: HIVE-21371: Make NonSyncByteArrayOutputStream Overflow Conscious (David Mollitor, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git commit c3939dab9c7d9429464add3e95741de76e78a3ab Author: David Mollitor AuthorDate: Sun Mar 10 22:50:44 2019 -0700 HIVE-21371: Make NonSyncByteArrayOutputStream Overflow Conscious (David Mollitor, reviewed by Gopal V) Signed-off-by: Gopal V --- .../common/io/NonSyncByteArrayOutputStream.java| 37 +++--- 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/common/io/NonSyncByteArrayOutputStream.java b/common/src/java/org/apache/hadoop/hive/common/io/NonSyncByteArrayOutputStream.java index ea49231..7e09cc8 100644 --- a/common/src/java/org/apache/hadoop/hive/common/io/NonSyncByteArrayOutputStream.java +++ b/common/src/java/org/apache/hadoop/hive/common/io/NonSyncByteArrayOutputStream.java @@ -21,12 +21,22 @@ import java.io.ByteArrayOutputStream; import java.io.DataInput; import java.io.IOException; import java.io.OutputStream; +import java.util.Arrays; /** * A thread-not-safe version of ByteArrayOutputStream, which removes all * synchronized modifiers. */ public class NonSyncByteArrayOutputStream extends ByteArrayOutputStream { + + /** + * The maximum size of array to allocate. + * Some VMs reserve some header words in an array. + * Attempts to allocate larger arrays may result in + * OutOfMemoryError: Requested array size exceeds VM limit + */ + private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8; + public NonSyncByteArrayOutputStream(int size) { super(size); } @@ -71,18 +81,22 @@ public class NonSyncByteArrayOutputStream extends ByteArrayOutputStream { count += 1; } - private int enLargeBuffer(int increment) { -int temp = count + increment; -int newLen = temp; -if (temp > buf.length) { - if ((buf.length << 1) > temp) { -newLen = buf.length << 1; + private void enLargeBuffer(final int increment) { +final int requestCapacity = Math.addExact(count, increment); +final int currentCapacity = buf.length; + +if (requestCapacity > currentCapacity) { + // Increase size by a factor of 1.5x + int newCapacity = currentCapacity + (currentCapacity >> 1); + + // Check for overflow scenarios + if (newCapacity < 0 || newCapacity > MAX_ARRAY_SIZE) { +newCapacity = MAX_ARRAY_SIZE; + } else if (newCapacity < requestCapacity) { +newCapacity = requestCapacity; } - byte newbuf[] = new byte[newLen]; - System.arraycopy(buf, 0, newbuf, 0, count); - buf = newbuf; + buf = Arrays.copyOf(buf, newCapacity); } -return newLen; } /** @@ -93,7 +107,8 @@ public class NonSyncByteArrayOutputStream extends ByteArrayOutputStream { if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) || ((off + len) < 0)) { throw new IndexOutOfBoundsException(); -} else if (len == 0) { +} +if (len == 0) { return; } enLargeBuffer(len);
[hive] branch master updated: HIVE-21294: Vectorization: 1-reducer Shuffle can skip the object hash functions (Teddy Choi, reviewed by Gopal V)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 84f766e HIVE-21294: Vectorization: 1-reducer Shuffle can skip the object hash functions (Teddy Choi, reviewed by Gopal V) 84f766e is described below commit 84f766e79f4d28dbd5b59067c40741fbff297aa7 Author: Teddy Choi AuthorDate: Wed Mar 6 17:10:32 2019 -0800 HIVE-21294: Vectorization: 1-reducer Shuffle can skip the object hash functions (Teddy Choi, reviewed by Gopal V) Signed-off-by: Gopal V --- .../VectorReduceSinkObjectHashOperator.java| 65 ++ 1 file changed, 28 insertions(+), 37 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java index 767df21..ef5ca02 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java @@ -64,6 +64,8 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp protected int[] reduceSinkPartitionColumnMap; protected TypeInfo[] reduceSinkPartitionTypeInfos; + private boolean isSingleReducer; + protected VectorExpression[] reduceSinkPartitionExpressions; // The above members are initialized by the constructor and must not be @@ -119,6 +121,8 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp reduceSinkPartitionTypeInfos = vectorReduceSinkInfo.getReduceSinkPartitionTypeInfos(); reduceSinkPartitionExpressions = vectorReduceSinkInfo.getReduceSinkPartitionExpressions(); } + +isSingleReducer = this.conf.getNumReducers() == 1; } private ObjectInspector[] getObjectInspectorArray(TypeInfo[] typeInfos) { @@ -255,48 +259,35 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp final int size = batch.size; - if (isEmptyBuckets) { // EmptyBuckets = true -if (isEmptyPartitions) { // isEmptyPartition = true - for (int logical = 0; logical< size; logical++) { -final int batchIndex = (selectedInUse ? selected[logical] : logical); -final int hashCode = nonPartitionRandom.nextInt(); -postProcess(batch, batchIndex, tag, hashCode); - } -} else { // isEmptyPartition = false - for (int logical = 0; logical< size; logical++) { -final int batchIndex = (selectedInUse ? selected[logical] : logical); -partitionVectorExtractRow.extractRow(batch, batchIndex, partitionFieldValues); -final int hashCode = hashFunc.apply(partitionFieldValues, partitionObjectInspectors); -postProcess(batch, batchIndex, tag, hashCode); + for (int logical = 0; logical< size; logical++) { +final int batchIndex = (selectedInUse ? selected[logical] : logical); +int hashCode; +if (isEmptyPartitions) { + if (isSingleReducer) { +// Empty partition, single reducer -> constant hashCode +hashCode = 0; + } else { +// Empty partition, multiple reducers -> random hashCode +hashCode = nonPartitionRandom.nextInt(); } +} else { + // Compute hashCode from partitions + partitionVectorExtractRow.extractRow(batch, batchIndex, partitionFieldValues); + hashCode = hashFunc.apply(partitionFieldValues, partitionObjectInspectors); } - } else { // EmptyBuckets = false -if (isEmptyPartitions) { // isEmptyPartition = true - for (int logical = 0; logical< size; logical++) { -final int batchIndex = (selectedInUse ? selected[logical] : logical); -bucketVectorExtractRow.extractRow(batch, batchIndex, bucketFieldValues); -final int bucketNum = ObjectInspectorUtils.getBucketNumber( - hashFunc.apply(bucketFieldValues, bucketObjectInspectors), numBuckets); -final int hashCode = nonPartitionRandom.nextInt() * 31 + bucketNum; -if (bucketExpr != null) { - evaluateBucketExpr(batch, batchIndex, bucketNum); -} -postProcess(batch, batchIndex, tag, hashCode); - } -} else { // isEmptyPartition = false - for (int logical = 0; logical< size; logical++) { -final int batchIndex = (selectedInUse ? selected[logical] : logical); -partitionVectorExtractRow.extractRow(batch, batchIndex, partitionFieldValues); -bucketVectorExtractRow.extractRow(batch, batchIndex, bucketFieldValues); -
[hive] branch master updated: HIVE-21232: LLAP: Add a cache-miss friendly split affinity provider (Gopal V, reviewed by Slim Bouguerra)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new c45751f HIVE-21232: LLAP: Add a cache-miss friendly split affinity provider (Gopal V, reviewed by Slim Bouguerra) c45751f is described below commit c45751fb4029009e08f6389c21a68c3ba26ec6de Author: Gopal V AuthorDate: Thu Feb 21 17:04:26 2019 -0800 HIVE-21232: LLAP: Add a cache-miss friendly split affinity provider (Gopal V, reviewed by Slim Bouguerra) --- .../java/org/apache/hadoop/hive/conf/HiveConf.java | 6 +-- .../tez/HostAffinitySplitLocationProvider.java | 45 ++ .../tez/TestHostAffinitySplitLocationProvider.java | 42 3 files changed, 83 insertions(+), 10 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 11f165a..04166db 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -4352,9 +4352,9 @@ public class HiveConf extends Configuration { "Whether LLAP daemon web UI should use SSL.", "llap.daemon.service.ssl"), LLAP_CLIENT_CONSISTENT_SPLITS("hive.llap.client.consistent.splits", true, "Whether to setup split locations to match nodes on which llap daemons are running, " + -"instead of using the locations provided by the split itself. If there is no llap daemon " + -"running, fall back to locations provided by the split. This is effective only if " + -"hive.execution.mode is llap"), +"preferring one of the locations provided by the split itself. If there is no llap daemon " + +"running on any of those locations (or on the cloud), fall back to a cache affinity to" + +" an LLAP node. This is effective only if hive.execution.mode is llap."), LLAP_VALIDATE_ACLS("hive.llap.validate.acls", true, "Whether LLAP should reject permissive ACLs in some cases (e.g. its own management\n" + "protocol or ZK paths), similar to how ssh refuses a key with bad access permissions."), diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java index c5d96e5..5224429 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HostAffinitySplitLocationProvider.java @@ -15,11 +15,11 @@ package org.apache.hadoop.hive.ql.exec.tez; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.List; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.hash.Hashing; +import java.util.Set; import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.mapred.FileSplit; @@ -29,6 +29,10 @@ import org.apache.hive.common.util.Murmur3; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.hash.Hashing; + /** * This maps a split (path + offset) to an index based on the number of locations provided. * @@ -47,18 +51,20 @@ public class HostAffinitySplitLocationProvider implements SplitLocationProvider private final boolean isDebugEnabled = LOG.isDebugEnabled(); private final List locations; + private final Set locationSet; public HostAffinitySplitLocationProvider(List knownLocations) { Preconditions.checkState(knownLocations != null && !knownLocations.isEmpty(), HostAffinitySplitLocationProvider.class.getName() + " needs at least 1 location to function"); this.locations = knownLocations; +this.locationSet = new HashSet(knownLocations); } @Override public String[] getLocations(InputSplit split) throws IOException { if (!(split instanceof FileSplit)) { - if (LOG.isDebugEnabled()) { + if (isDebugEnabled) { LOG.debug("Split: " + split + " is not a FileSplit. Using default locations"); } return split.getLocations(); @@ -66,14 +72,39 @@ public class HostAffinitySplitLocationProvider implements SplitLocationProvider FileSplit fsplit = (FileSplit) split; String splitDesc = "Split at " + fsplit.getPath() + " with offset= " + fsplit.getStart() + ", length=" + fsplit.getLength(); -String location = locations.get(determineLoc
[hive] branch master updated: HIVE-21233: ACID: Make validate_acid_sort_order() Stateful UDF (Gopal V, reviewed by Vaibhav Gumashta)
This is an automated email from the ASF dual-hosted git repository. gopalv pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git The following commit(s) were added to refs/heads/master by this push: new 4ddc9de HIVE-21233: ACID: Make validate_acid_sort_order() Stateful UDF (Gopal V, reviewed by Vaibhav Gumashta) 4ddc9de is described below commit 4ddc9de90b6de032d77709c9631ab787cef225d5 Author: Gopal V AuthorDate: Fri Feb 8 12:28:23 2019 -0800 HIVE-21233: ACID: Make validate_acid_sort_order() Stateful UDF (Gopal V, reviewed by Vaibhav Gumashta) --- .../udf/generic/GenericUDFValidateAcidSortOrder.java | 19 +++ 1 file changed, 19 insertions(+) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFValidateAcidSortOrder.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFValidateAcidSortOrder.java index 757a366..10e279e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFValidateAcidSortOrder.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFValidateAcidSortOrder.java @@ -1,9 +1,27 @@ +/* + * 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.io.BucketCodec; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.UDFType; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter; @@ -14,6 +32,7 @@ import org.apache.hadoop.io.LongWritable; /** * GenericUDFValidateAcidSortOrder. */ +@UDFType(stateful=true) @Description(name = "validate_acid_sort_order(ROW__ID.writeId, ROW__ID.bucketId, ROW__ID.rowId)", value = "_FUNC_(writeId, bucketId, rowId) - returns 0 if the current row is in the right acid sort order " + "compared to the previous row")
hive git commit: HIVE-21005: LLAP: Reading more stripes per-split leaks ZlibCodecs (Nita Dembla, reviewed by Gopal V)
Repository: hive Updated Branches: refs/heads/master 9925eb108 -> b101af1d0 HIVE-21005: LLAP: Reading more stripes per-split leaks ZlibCodecs (Nita Dembla, reviewed by Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b101af1d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b101af1d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b101af1d Branch: refs/heads/master Commit: b101af1d0563fe2bbe41f5e9f38ea575e1e3237c Parents: 9925eb1 Author: Nita Dembla Authored: Tue Dec 11 10:34:43 2018 -0800 Committer: Gopal V Committed: Tue Dec 11 10:34:43 2018 -0800 -- .../hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java | 7 +++ 1 file changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b101af1d/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java -- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java index 74cee64..e6d8b7a 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java @@ -434,6 +434,13 @@ public class OrcEncodedDataReader extends CallableWithNdc private void ensureDataReader() throws IOException { ensureOrcReader(); +if (stripeReader != null) { + try { +stripeReader.close(); + } catch (IOException ex) { +// Ignore. + } +} // Reader creation updates HDFS counters, don't do it here. DataWrapperForOrc dw = new DataWrapperForOrc(); stripeReader = orcReader.encodedReader(
hive git commit: HIVE-20490: UDAF: Add an 'approx_distinct' to Hive (Gopal V, reviewed by Gunther Hagleitner, Nishant Bangarwa)
Repository: hive Updated Branches: refs/heads/master 5eebbdf7c -> 7b9540e48 HIVE-20490: UDAF: Add an 'approx_distinct' to Hive (Gopal V, reviewed by Gunther Hagleitner, Nishant Bangarwa) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7b9540e4 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7b9540e4 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7b9540e4 Branch: refs/heads/master Commit: 7b9540e48be1d896c8229564e49f5803bbca6a27 Parents: 5eebbdf Author: Gopal V Authored: Sun Oct 21 13:44:33 2018 -0700 Committer: Gopal V Committed: Sun Oct 21 13:44:33 2018 -0700 -- .../test/resources/testconfiguration.properties | 3 +- .../hadoop/hive/ql/exec/FunctionRegistry.java | 2 + .../generic/GenericUDAFApproximateDistinct.java | 237 + .../queries/clientpositive/approx_distinct.q| 76 +++ .../clientpositive/llap/approx_distinct.q.out | 490 +++ .../results/clientpositive/show_functions.q.out | 1 + 6 files changed, 808 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7b9540e4/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 8349e3d..ff9f758 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -938,7 +938,8 @@ minillaplocal.query.files=\ partialdhj.q,\ stats_date.q,\ dst.q,\ - q93_with_constraints.q + q93_with_constraints.q,\ + approx_distinct.q encrypted.query.files=encryption_join_unencrypted_tbl.q,\ encryption_insert_partition_static.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/7b9540e4/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java -- 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 00e97a0..578b16c 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 @@ -68,6 +68,7 @@ import org.apache.hadoop.hive.ql.udf.UDFFromUnixTime; import org.apache.hadoop.hive.ql.udf.UDFHex; import org.apache.hadoop.hive.ql.udf.UDFHour; import org.apache.hadoop.hive.ql.udf.UDFJson; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFApproximateDistinct; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLength; import org.apache.hadoop.hive.ql.udf.UDFLike; import org.apache.hadoop.hive.ql.udf.UDFLn; @@ -465,6 +466,7 @@ public final class FunctionRegistry { system.registerGenericUDAF("compute_stats", new GenericUDAFComputeStats()); system.registerGenericUDAF("bloom_filter", new GenericUDAFBloomFilter()); +system.registerGenericUDAF("approx_distinct", new GenericUDAFApproximateDistinct()); system.registerUDAF("percentile", UDAFPercentile.class); http://git-wip-us.apache.org/repos/asf/hive/blob/7b9540e4/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFApproximateDistinct.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFApproximateDistinct.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFApproximateDistinct.java new file mode 100644 index 000..1e8fc8a --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFApproximateDistinct.java @@ -0,0 +1,237 @@ +/* + * 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 java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; + +import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog; +import org.apache.hadoop.hive.common.ndv.hll.H
hive git commit: HIVE-20768: Adding Tumbling Window UDF (Slim Bouguerra, via Gopal V)
Repository: hive Updated Branches: refs/heads/master ccdc92501 -> 0d39a81c1 HIVE-20768: Adding Tumbling Window UDF (Slim Bouguerra, via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0d39a81c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0d39a81c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0d39a81c Branch: refs/heads/master Commit: 0d39a81c1177093312133504f5026f63fe06d08a Parents: ccdc925 Author: Slim Bouguerra Authored: Sun Oct 21 11:40:54 2018 -0700 Committer: Gopal V Committed: Sun Oct 21 11:40:54 2018 -0700 -- .../hadoop/hive/ql/exec/FunctionRegistry.java | 1 + .../ql/udf/generic/GenericUDFTumbledWindow.java | 139 +++ .../clientpositive/tumbled_window_tests.q | 19 +++ .../results/clientpositive/show_functions.q.out | 1 + .../clientpositive/tumbled_window_tests.q.out | 135 ++ 5 files changed, 295 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/0d39a81c/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java -- 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 b7ca7c7..00e97a0 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 @@ -517,6 +517,7 @@ public final class FunctionRegistry { system.registerGenericUDF("to_epoch_milli", GenericUDFEpochMilli.class); system.registerGenericUDF("bucket_number", GenericUDFBucketNumber.class); +system.registerGenericUDF("tumbling_window", GenericUDFTumbledWindow.class); // Generic UDTF's system.registerGenericUDTF("explode", GenericUDTFExplode.class); http://git-wip-us.apache.org/repos/asf/hive/blob/0d39a81c/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTumbledWindow.java -- 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 new file mode 100644 index 000..17823c0 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTumbledWindow.java @@ -0,0 +1,139 @@ +/* + * 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.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.common.type.Timestamp; +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.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +/** + * Tumbling windows are a series of fixed-sized, non-overlapping and contiguous time intervals. + * Tumbling windows are inclusive start exclusive end. + * By default the beginning instant of fist window is Epoch 0 Thu Jan 01 00:00:00 1970 UTC. + * Optionally users may provide a different origin as a timestamp arg3. + * + * This an example of series of window with an interval of 5 seconds and origin Epoch 0 Thu Jan 01 00:00:00 1970 UTC: + * + * + * interval 1 interval 2interval 3 + * Jan 01 00:00:00 Jan 01 00:00:05 Jan 01 00:00:10 + * 0 -- 4 : 5 --- 9: 10 --- 14 + * + * This UDF rounds timestamp agr1 to the beginning of window interval where it belongs to. + * +
hive git commit: HIVE-20338: LLAP: Force synthetic file-id for filesystems which have HDFS protocol impls with POSIX mutation semantics (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master d9d431cfe -> 240bfb464 HIVE-20338: LLAP: Force synthetic file-id for filesystems which have HDFS protocol impls with POSIX mutation semantics (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/240bfb46 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/240bfb46 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/240bfb46 Branch: refs/heads/master Commit: 240bfb464631ddb521406eac2348968bbee16c51 Parents: d9d431c Author: Gopal V Authored: Wed Oct 3 12:22:10 2018 -0700 Committer: Gopal V Committed: Wed Oct 3 13:11:53 2018 -0700 -- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java| 3 ++- .../hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java| 8 +--- .../hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java | 7 --- ql/src/java/org/apache/hadoop/hive/ql/io/HdfsUtils.java | 4 ++-- .../org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java | 7 +-- .../ql/io/parquet/vector/VectorizedParquetRecordReader.java | 3 ++- 6 files changed, 20 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/240bfb46/common/src/java/org/apache/hadoop/hive/conf/HiveConf.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 a49f8af..531fabd 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3942,7 +3942,8 @@ public class HiveConf extends Configuration { "is unneeded. This is only necessary for ORC files written before HIVE-9660."), LLAP_IO_USE_FILEID_PATH("hive.llap.io.use.fileid.path", true, "Whether LLAP should use fileId (inode)-based path to ensure better consistency for the\n" + -"cases of file overwrites. This is supported on HDFS."), +"cases of file overwrites. This is supported on HDFS. Disabling this also turns off any\n" + +"cache consistency checks based on fileid comparisons."), // Restricted to text for now as this is a new feature; only text files can be sliced. LLAP_IO_ENCODE_ENABLED("hive.llap.io.encode.enabled", true, "Whether LLAP should try to re-encode and cache data for non-ORC formats. This is used\n" + http://git-wip-us.apache.org/repos/asf/hive/blob/240bfb46/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java -- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java index e8a3b40..4f5b0a9 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java @@ -214,7 +214,9 @@ public class OrcEncodedDataReader extends CallableWithNdc fs = split.getPath().getFileSystem(jobConf); fileKey = determineFileId(fs, split, HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_CACHE_ALLOW_SYNTHETIC_FILEID), -HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_CACHE_DEFAULT_FS_FILE_ID)); +HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_CACHE_DEFAULT_FS_FILE_ID), +!HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_IO_USE_FILEID_PATH) +); fileMetadata = getFileFooterFromCacheOrDisk(); final TypeDescription fileSchema = fileMetadata.getSchema(); @@ -464,7 +466,7 @@ public class OrcEncodedDataReader extends CallableWithNdc } private static Object determineFileId(FileSystem fs, FileSplit split, - boolean allowSynthetic, boolean checkDefaultFs) throws IOException { + boolean allowSynthetic, boolean checkDefaultFs, boolean forceSynthetic) throws IOException { if (split instanceof OrcSplit) { Object fileKey = ((OrcSplit)split).getFileKey(); if (fileKey != null) { @@ -472,7 +474,7 @@ public class OrcEncodedDataReader extends CallableWithNdc } } LOG.warn("Split for " + split.getPath() + " (" + split.getClass() + ") does not have file ID"); -return HdfsUtils.getFileId(fs, split.getPath(), allowSynthetic, checkDefaultFs); +return HdfsUtils.getFileId(fs, split.getPath(), allowSynthetic, checkDefaultFs, forceSynthetic); } /** http://git-wip-us.apache.org/repos/asf/hive/blob/240bfb46/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java -- diff --git a/llap-server
[2/4] hive git commit: HIVE-20377: Hive Kafka Storage Handler (Slim Bouguerra via Gopal Vijayaraghavan)
http://git-wip-us.apache.org/repos/asf/hive/blob/48f92c31/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java -- diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java new file mode 100644 index 000..5de51cd --- /dev/null +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -0,0 +1,312 @@ +/* + * 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.kafka; + +import com.google.common.collect.ImmutableList; +import kafka.admin.AdminUtils; +import kafka.admin.RackAwareMode; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServer; +import kafka.utils.MockTime; +import kafka.utils.TestUtils; +import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; +import kafka.zk.EmbeddedZookeeper; +import org.I0Itec.zkclient.ZkClient; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.utils.Time; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Kafka Iterator Tests. + */ +public class KafkaRecordIteratorTest { + private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordIteratorTest.class); + private static final int RECORD_NUMBER = 100; + private static final String TOPIC = "my_test_topic"; + private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, 0); + public static final byte[] KEY_BYTES = "KEY".getBytes(Charset.forName("UTF-8")); + private static final List> + RECORDS = + IntStream.range(0, RECORD_NUMBER).mapToObj(number -> { +final byte[] value = ("VALUE-" + Integer.toString(number)).getBytes(Charset.forName("UTF-8")); +return new ConsumerRecord<>(TOPIC, 0, (long) number, 0L, null, 0L, 0, 0, KEY_BYTES, value); + }).collect(Collectors.toList()); + public static final long POLL_TIMEOUT_MS = 900L; + private static ZkUtils zkUtils; + private static ZkClient zkClient; + private static KafkaProducer producer; + private static KafkaServer kafkaServer; + private static String zkConnect; + private KafkaConsumer consumer = null; + private KafkaRecordIterator kafkaRecordIterator = null; + private Configuration conf = new Configuration(); + private static EmbeddedZookeeper zkServer; + + public KafkaRecordIteratorTest() { + } + + @BeforeClass public static void setupCluster() throws IOException { +LOG.info("init embedded Zookeeper"); +zkServer = new EmbeddedZookeeper(); +zkConnect = "127.0.0.1:" + zkServer.port(); +zkClient = new ZkClient(zkConnect, 3000, 3000, ZKStringSerializer$.MODULE$); +zkUtils = ZkUtils.apply(zkClient, false); +LOG.info("init kafka broker"); +Properties brokerProps = new Properties(); +brokerProps.setProperty("zookeeper.connect", zkConnect); +brokerProps.setProperty("broker.id", "0"); +brokerProps.setProperty("log.dir", Files.createTempDirectory("kafka-log-dir-").toAbsolutePath().toString()); +brokerProps.setProper
[4/4] hive git commit: HIVE-20377: Hive Kafka Storage Handler (Slim Bouguerra via Gopal Vijayaraghavan)
HIVE-20377: Hive Kafka Storage Handler (Slim Bouguerra via Gopal Vijayaraghavan) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/48f92c31 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/48f92c31 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/48f92c31 Branch: refs/heads/master Commit: 48f92c31dee3983f573f2e66baaa213a0196f1ba Parents: 33fa62f Author: Slim Bouguerra Authored: Tue Sep 4 21:05:06 2018 -0700 Committer: Gopal V Committed: Tue Sep 4 22:20:21 2018 -0700 -- itests/qtest-druid/pom.xml |2 +- .../hive/kafka/SingleNodeKafkaCluster.java | 41 +- .../java/org/apache/hive/kafka/Wikipedia.java | 1418 ++ itests/qtest/pom.xml|7 +- .../hive/cli/TestMiniDruidKafkaCliDriver.java |2 - .../test/resources/testconfiguration.properties |3 +- .../hadoop/hive/cli/control/CliConfigs.java |1 + .../org/apache/hadoop/hive/ql/QTestUtil.java| 56 +- kafka-handler/pom.xml | 160 ++ .../hadoop/hive/kafka/GenericKafkaSerDe.java| 192 +++ .../hadoop/hive/kafka/KafkaJsonSerDe.java | 248 +++ .../hive/kafka/KafkaPullerInputFormat.java | 205 +++ .../hive/kafka/KafkaPullerInputSplit.java | 213 +++ .../hive/kafka/KafkaPullerRecordReader.java | 164 ++ .../hadoop/hive/kafka/KafkaRecordIterator.java | 209 +++ .../hadoop/hive/kafka/KafkaRecordWritable.java | 176 +++ .../hadoop/hive/kafka/KafkaScanTrimmer.java | 482 ++ .../hadoop/hive/kafka/KafkaStorageHandler.java | 138 ++ .../hadoop/hive/kafka/KafkaStreamingUtils.java | 175 +++ .../apache/hadoop/hive/kafka/package-info.java | 23 + .../hive/kafka/KafkaPullerInputSplitTest.java | 131 ++ .../hive/kafka/KafkaRecordIteratorTest.java | 312 .../hive/kafka/KafkaRecordWritableTest.java | 52 + .../hadoop/hive/kafka/KafkaScanTrimmerTest.java | 569 +++ .../hive/kafka/KafkaStreamingUtilsTest.java | 59 + .../apache/hadoop/hive/kafka/package-info.java | 23 + .../hadoop/hive/llap/cli/LlapServiceDriver.java | 87 +- packaging/pom.xml |5 + pom.xml |5 +- .../hive/ql/index/IndexPredicateAnalyzer.java | 24 +- .../clientpositive/kafka_storage_handler.q | 236 +++ .../druid/kafka_storage_handler.q.out | 845 +++ .../serde2/avro/AvroLazyObjectInspector.java|6 +- 33 files changed, 6199 insertions(+), 70 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/48f92c31/itests/qtest-druid/pom.xml -- diff --git a/itests/qtest-druid/pom.xml b/itests/qtest-druid/pom.xml index 79a0fb3..e566fcf 100644 --- a/itests/qtest-druid/pom.xml +++ b/itests/qtest-druid/pom.xml @@ -43,7 +43,7 @@ 10.11.1.1 16.0.1 4.1.0 -0.10.2.0 +1.0.1 http://git-wip-us.apache.org/repos/asf/hive/blob/48f92c31/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java -- diff --git a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java index d839fd2..c9339b5 100644 --- a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java +++ b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java @@ -10,6 +10,7 @@ import kafka.utils.ZkUtils; import org.apache.hadoop.service.AbstractService; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; import com.google.common.base.Throwables; @@ -25,6 +26,7 @@ import java.io.IOException; import java.nio.charset.Charset; import java.util.List; import java.util.Properties; +import java.util.stream.IntStream; /** * This class has the hooks to start and stop single node kafka cluster. @@ -32,6 +34,10 @@ import java.util.Properties; */ public class SingleNodeKafkaCluster extends AbstractService { private static final Logger log = LoggerFactory.getLogger(SingleNodeKafkaCluster.class); + private static final int BROKER_PORT = 9092; + private static final String LOCALHOST = "localhost"; + private static final String LOCALHOST_9092 = String.format("%s:%s", LOCALHOST, BROKER_PORT); + private final KafkaServerStartable serverStartable; private final String zkString; @@ -42,9 +48,12 @@ public class SingleNodeKafkaCluster extends
[1/4] hive git commit: HIVE-20377: Hive Kafka Storage Handler (Slim Bouguerra via Gopal Vijayaraghavan)
Repository: hive Updated Branches: refs/heads/master 33fa62f18 -> 48f92c31d http://git-wip-us.apache.org/repos/asf/hive/blob/48f92c31/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out -- diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out new file mode 100644 index 000..3dec33d --- /dev/null +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -0,0 +1,845 @@ +PREHOOK: query: CREATE EXTERNAL TABLE kafka_table +(`__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +WITH SERDEPROPERTIES ("timestamp.formats"="-MM-dd\'T\'HH:mm:ss\'Z\'") +TBLPROPERTIES +("kafka.topic" = "test-topic", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe") +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@kafka_table +POSTHOOK: query: CREATE EXTERNAL TABLE kafka_table +(`__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +WITH SERDEPROPERTIES ("timestamp.formats"="-MM-dd\'T\'HH:mm:ss\'Z\'") +TBLPROPERTIES +("kafka.topic" = "test-topic", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe") +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@kafka_table +PREHOOK: query: DESCRIBE EXTENDED kafka_table +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@kafka_table +POSTHOOK: query: DESCRIBE EXTENDED kafka_table +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@kafka_table +__time timestamp from deserializer +page string from deserializer +user string from deserializer +language string from deserializer +countrystring from deserializer +continent string from deserializer +namespace string from deserializer +newpageboolean from deserializer +unpatrolledboolean from deserializer +anonymous boolean from deserializer +robot boolean from deserializer +added int from deserializer +deletedint from deserializer +delta bigint from deserializer +__partitionint from deserializer +__offset bigint from deserializer +__timestampbigint from deserializer +__start_offset bigint from deserializer +__end_offset bigint from deserializer + + A masked pattern was here +PREHOOK: query: Select `__partition` ,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition` ,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 0 10 0 NULLGypsy Dangernuclear en United States North America article truetruefalse false 57 200 -143 +0 0 10 1 NULLStriker Eureka speed en Australia Australia wikipedia truefalse false true 459 129 330 +0 0 10 2 NULLCherno AlphamasterYiru Russia Asiaarticle truefalse false true123 12 111 +0 0 10 3 NULLCrimson Typhoon tripletszh China Asiawikipedia false truefalse true905 5 900 +0 0 10 4 NULLCoyote Tango
[3/4] hive git commit: HIVE-20377: Hive Kafka Storage Handler (Slim Bouguerra via Gopal Vijayaraghavan)
http://git-wip-us.apache.org/repos/asf/hive/blob/48f92c31/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java -- diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java new file mode 100644 index 000..2d5637d --- /dev/null +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -0,0 +1,205 @@ +/* + * 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.kafka; + +import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * Kafka puller input format to read records from a Kafka Queue. + * The input split will contain the set of topic partition and start/end offsets. + * Records will be returned as bytes array. + */ +public class KafkaPullerInputFormat extends InputFormat +implements org.apache.hadoop.mapred.InputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaPullerInputFormat.class); + + @Override public InputSplit[] getSplits(JobConf jobConf, int i) throws IOException { +List inputSplits; +try { + inputSplits = computeSplits(jobConf); +} catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); +} +InputSplit[] inputSplitsArray = new InputSplit[inputSplits.size()]; +return inputSplits.toArray(inputSplitsArray); + } + + /** + * Build a full scan using Kafka list partition then beginning/end offsets. + * This function might block duo to calls like: + * org.apache.kafka.clients.consumer.KafkaConsumer#beginningOffsets(java.util.Collection) + * + * @param topic kafka topic + * @param consumer initialized kafka consumer + * @param tablePaths hive table path + * + * @return full scan input split collection based on Kafka metadata APIs + */ + private static List buildFullScanFromKafka(String topic, + KafkaConsumer consumer, + Path[] tablePaths) { +final Map starOffsetsMap; +final Map endOffsetsMap; + +final List topicPartitions; +topicPartitions = fetchTopicPartitions(topic, consumer); +starOffsetsMap = consumer.beginningOffsets(topicPartitions); +endOffsetsMap = consumer.endOffsets(topicPartitions); + +if (LOG.isDebugEnabled()) { + LOG.info("Found the following partitions [{}]", + topicPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(","))); + starOffsetsMap.forEach((tp, start) -> LOG.info("TPartition [{}],Start offsets [{}]", tp, start)); + endOffsetsMap.forEach((tp, end) -> LOG.info("TPartition [{}],End offsets [{}]", tp, end)); +} +return topicPartitions.stream() +.map(topicPartition -> new KafkaPullerInputSplit(topicPartition.topic(), +topicPartition.partition(), +starO
hive git commit: HIVE-20395. Parallelize files move in 'replaceFiles' method. (Igor Dvorzhak, reviewed by Gopal Vijayaraghavan)
Repository: hive Updated Branches: refs/heads/master df1a02be3 -> 33fa62f18 HIVE-20395. Parallelize files move in 'replaceFiles' method. (Igor Dvorzhak, reviewed by Gopal Vijayaraghavan) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/33fa62f1 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/33fa62f1 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/33fa62f1 Branch: refs/heads/master Commit: 33fa62f18849f130fb14727d390888e7b0e5d09d Parents: df1a02b Author: Igor Dvorzhak Authored: Tue Sep 4 15:01:45 2018 -0700 Committer: Gopal V Committed: Tue Sep 4 15:01:58 2018 -0700 -- .../apache/hadoop/hive/ql/metadata/Hive.java| 41 +++- 1 file changed, 39 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/33fa62f1/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.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 9cb5717..0b1048c 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 @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import static org.apache.hadoop.hive.conf.Constants.MATERIALIZED_VIEW_REWRITING_TIME_WINDOW; @@ -4362,16 +4363,52 @@ private void constructOneLBLocationMap(FileStatus fSta, listNewFilesRecursively(destFs, destf, newFiles); } } else { +final Map, Path> moveFutures = Maps.newLinkedHashMapWithExpectedSize(srcs.length); +final int moveFilesThreadCount = HiveConf.getIntVar(conf, ConfVars.HIVE_MOVE_FILES_THREAD_COUNT); +final ExecutorService pool = moveFilesThreadCount > 0 +? Executors.newFixedThreadPool( +moveFilesThreadCount, +new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Replace-Thread-%d").build()) +: MoreExecutors.newDirectExecutorService(); +final SessionState parentSession = SessionState.get(); // its either a file or glob for (FileStatus src : srcs) { Path destFile = new Path(destf, src.getPath().getName()); - if (!moveFile(conf, src.getPath(), destFile, true, isSrcLocal, isManaged)) { + moveFutures.put( + pool.submit( + new Callable() { +@Override +public Boolean call() throws Exception { + SessionState.setCurrentSessionState(parentSession); + return moveFile( + conf, src.getPath(), destFile, true, isSrcLocal, isManaged); +} + }), + destFile); +} + +pool.shutdown(); +for (Map.Entry, Path> moveFuture : moveFutures.entrySet()) { + boolean moveFailed; + try { +moveFailed = !moveFuture.getKey().get(); + } catch (InterruptedException | ExecutionException e) { +pool.shutdownNow(); +if (e.getCause() instanceof IOException) { + throw (IOException) e.getCause(); +} +if (e.getCause() instanceof HiveException) { + throw (HiveException) e.getCause(); +} +throw handlePoolException(pool, e); + } + if (moveFailed) { throw new IOException("Error moving: " + srcf + " into: " + destf); } // Add file paths of the files that will be moved to the destination if the caller needs it if (null != newFiles) { -newFiles.add(destFile); +newFiles.add(moveFuture.getValue()); } } }
hive git commit: HIVE-20101: BloomKFilter: Avoid using the local byte[] arrays entirely (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/branch-3 d999a3b23 -> 7774508f5 HIVE-20101: BloomKFilter: Avoid using the local byte[] arrays entirely (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7774508f Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7774508f Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7774508f Branch: refs/heads/branch-3 Commit: 7774508f50db2bbda8338603a163dd29f26d057a Parents: d999a3b Author: Gopal V Authored: Tue Jul 31 14:10:06 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 14:12:04 2018 -0700 -- .../apache/hive/common/util/BloomKFilter.java | 30 +++- 1 file changed, 4 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7774508f/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java -- diff --git a/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java b/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java index 5b1914d..3b44d2b 100644 --- a/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java +++ b/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java @@ -36,8 +36,6 @@ import java.util.Arrays; * This implementation has much lesser L1 data cache misses than {@link BloomFilter}. */ public class BloomKFilter { - private final byte[] BYTE_ARRAY_4 = new byte[4]; - private final byte[] BYTE_ARRAY_8 = new byte[8]; public static final float DEFAULT_FPP = 0.05f; private static final int DEFAULT_BLOCK_SIZE = 8; private static final int DEFAULT_BLOCK_SIZE_BITS = (int) (Math.log(DEFAULT_BLOCK_SIZE) / Math.log(2)); @@ -149,8 +147,7 @@ public class BloomKFilter { } public void addInt(int val) { -// puts int in little endian order -addBytes(intToByteArrayLE(val)); +addHash(Murmur3.hash64(val)); } @@ -184,6 +181,7 @@ public class BloomKFilter { private boolean testHash(long hash64) { final int hash1 = (int) hash64; final int hash2 = (int) (hash64 >>> 32); +final long[] bits = bitSet.data; int firstHash = hash1 + hash2; // hashcode should be positive, flip all the bits if it's negative @@ -216,7 +214,7 @@ public class BloomKFilter { long expected = 0; for (int i = 0; i < DEFAULT_BLOCK_SIZE; i++) { final long mask = masks[i]; - expected |= (bitSet.data[blockBaseOffset + i] & mask) ^ mask; + expected |= (bits[blockBaseOffset + i] & mask) ^ mask; } // clear the mask for array reuse (this is to avoid masks array allocation in inner loop) @@ -235,7 +233,7 @@ public class BloomKFilter { } public boolean testInt(int val) { -return testBytes(intToByteArrayLE(val)); +return testHash(Murmur3.hash64(val)); } public boolean testLong(long val) { @@ -250,26 +248,6 @@ public class BloomKFilter { return testLong(Double.doubleToLongBits(val)); } - private byte[] intToByteArrayLE(int val) { -BYTE_ARRAY_4[0] = (byte) (val >> 0); -BYTE_ARRAY_4[1] = (byte) (val >> 8); -BYTE_ARRAY_4[2] = (byte) (val >> 16); -BYTE_ARRAY_4[3] = (byte) (val >> 24); -return BYTE_ARRAY_4; - } - - private byte[] longToByteArrayLE(long val) { -BYTE_ARRAY_8[0] = (byte) (val >> 0); -BYTE_ARRAY_8[1] = (byte) (val >> 8); -BYTE_ARRAY_8[2] = (byte) (val >> 16); -BYTE_ARRAY_8[3] = (byte) (val >> 24); -BYTE_ARRAY_8[4] = (byte) (val >> 32); -BYTE_ARRAY_8[5] = (byte) (val >> 40); -BYTE_ARRAY_8[6] = (byte) (val >> 48); -BYTE_ARRAY_8[7] = (byte) (val >> 56); -return BYTE_ARRAY_8; - } - public long sizeInBytes() { return getBitSize() / 8; }
hive git commit: HIVE-20101: BloomKFilter: Avoid using the local byte[] arrays entirely (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master 3e46515d3 -> 87ce36b45 HIVE-20101: BloomKFilter: Avoid using the local byte[] arrays entirely (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/87ce36b4 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/87ce36b4 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/87ce36b4 Branch: refs/heads/master Commit: 87ce36b458350db141c4cb4b6336a9a01796370f Parents: 3e46515 Author: Gopal V Authored: Tue Jul 31 14:10:06 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 14:10:06 2018 -0700 -- .../apache/hive/common/util/BloomKFilter.java | 30 +++- 1 file changed, 4 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/87ce36b4/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java -- diff --git a/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java b/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java index 5b1914d..3b44d2b 100644 --- a/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java +++ b/storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java @@ -36,8 +36,6 @@ import java.util.Arrays; * This implementation has much lesser L1 data cache misses than {@link BloomFilter}. */ public class BloomKFilter { - private final byte[] BYTE_ARRAY_4 = new byte[4]; - private final byte[] BYTE_ARRAY_8 = new byte[8]; public static final float DEFAULT_FPP = 0.05f; private static final int DEFAULT_BLOCK_SIZE = 8; private static final int DEFAULT_BLOCK_SIZE_BITS = (int) (Math.log(DEFAULT_BLOCK_SIZE) / Math.log(2)); @@ -149,8 +147,7 @@ public class BloomKFilter { } public void addInt(int val) { -// puts int in little endian order -addBytes(intToByteArrayLE(val)); +addHash(Murmur3.hash64(val)); } @@ -184,6 +181,7 @@ public class BloomKFilter { private boolean testHash(long hash64) { final int hash1 = (int) hash64; final int hash2 = (int) (hash64 >>> 32); +final long[] bits = bitSet.data; int firstHash = hash1 + hash2; // hashcode should be positive, flip all the bits if it's negative @@ -216,7 +214,7 @@ public class BloomKFilter { long expected = 0; for (int i = 0; i < DEFAULT_BLOCK_SIZE; i++) { final long mask = masks[i]; - expected |= (bitSet.data[blockBaseOffset + i] & mask) ^ mask; + expected |= (bits[blockBaseOffset + i] & mask) ^ mask; } // clear the mask for array reuse (this is to avoid masks array allocation in inner loop) @@ -235,7 +233,7 @@ public class BloomKFilter { } public boolean testInt(int val) { -return testBytes(intToByteArrayLE(val)); +return testHash(Murmur3.hash64(val)); } public boolean testLong(long val) { @@ -250,26 +248,6 @@ public class BloomKFilter { return testLong(Double.doubleToLongBits(val)); } - private byte[] intToByteArrayLE(int val) { -BYTE_ARRAY_4[0] = (byte) (val >> 0); -BYTE_ARRAY_4[1] = (byte) (val >> 8); -BYTE_ARRAY_4[2] = (byte) (val >> 16); -BYTE_ARRAY_4[3] = (byte) (val >> 24); -return BYTE_ARRAY_4; - } - - private byte[] longToByteArrayLE(long val) { -BYTE_ARRAY_8[0] = (byte) (val >> 0); -BYTE_ARRAY_8[1] = (byte) (val >> 8); -BYTE_ARRAY_8[2] = (byte) (val >> 16); -BYTE_ARRAY_8[3] = (byte) (val >> 24); -BYTE_ARRAY_8[4] = (byte) (val >> 32); -BYTE_ARRAY_8[5] = (byte) (val >> 40); -BYTE_ARRAY_8[6] = (byte) (val >> 48); -BYTE_ARRAY_8[7] = (byte) (val >> 56); -return BYTE_ARRAY_8; - } - public long sizeInBytes() { return getBitSize() / 8; }
[3/3] hive git commit: HIVE-19199: ACID: DbTxnManager heartbeat-service needs static sync init (Gopal V, reviewed by Eugene Koifman)
HIVE-19199: ACID: DbTxnManager heartbeat-service needs static sync init (Gopal V, reviewed by Eugene Koifman) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d999a3b2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d999a3b2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d999a3b2 Branch: refs/heads/branch-3 Commit: d999a3b23c1306712f91338efa8757b845114954 Parents: ca0d4f7 Author: Gopal V Authored: Tue Jul 31 13:51:53 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 13:53:22 2018 -0700 -- .../hadoop/hive/ql/lockmgr/DbTxnManager.java| 31 +++- 1 file changed, 17 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d999a3b2/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java -- 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 78980fa..9bc4d2a 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 @@ -978,21 +978,24 @@ public final class DbTxnManager extends HiveTxnManagerImpl { } private synchronized void initHeartbeatExecutorService() { -if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown() -&& !heartbeatExecutorService.isTerminated()) { - return; +synchronized (DbTxnManager.class) { + if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown() + && !heartbeatExecutorService.isTerminated()) { +return; + } + heartbeatExecutorService = + Executors.newScheduledThreadPool( + conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE), + new ThreadFactory() { +private final AtomicInteger threadCounter = new AtomicInteger(); + +@Override +public Thread newThread(Runnable r) { + return new HeartbeaterThread(r, "Heartbeater-" + threadCounter.getAndIncrement()); +} + }); + ((ScheduledThreadPoolExecutor) heartbeatExecutorService).setRemoveOnCancelPolicy(true); } -heartbeatExecutorService = -Executors.newScheduledThreadPool( - conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE), new ThreadFactory() { - private final AtomicInteger threadCounter = new AtomicInteger(); - - @Override - public Thread newThread(Runnable r) { -return new HeartbeaterThread(r, "Heartbeater-" + threadCounter.getAndIncrement()); - } -}); -((ScheduledThreadPoolExecutor) heartbeatExecutorService).setRemoveOnCancelPolicy(true); } public static class HeartbeaterThread extends Thread {
[2/3] hive git commit: HIVE-20040: JDBC: HTTP listen queue is 50 and SYNs are lost (Gopal V, reviewed by Prasanth Jayachandran)
HIVE-20040: JDBC: HTTP listen queue is 50 and SYNs are lost (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ca0d4f7c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ca0d4f7c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ca0d4f7c Branch: refs/heads/branch-3 Commit: ca0d4f7c55f5c3f885d3d178c7a012037036f15d Parents: 3bb4d1c Author: Gopal V Authored: Tue Jul 31 13:49:14 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 13:53:13 2018 -0700 -- .../org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java| 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ca0d4f7c/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java -- 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 95d78f8..89271d7 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 @@ -143,6 +143,7 @@ public class ThriftHttpCLIService extends ThriftCLIService { int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, TimeUnit.MILLISECONDS); connector.setIdleTimeout(maxIdleTime); + connector.setAcceptQueueSize(maxWorkerThreads); server.addConnector(connector);
[1/3] hive git commit: HIVE-20177: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode (Gopal V, reviewed by Matt McCline)
Repository: hive Updated Branches: refs/heads/branch-3 e96728c52 -> d999a3b23 HIVE-20177: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode (Gopal V, reviewed by Matt McCline) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3bb4d1ca Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3bb4d1ca Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3bb4d1ca Branch: refs/heads/branch-3 Commit: 3bb4d1cad063ece840d56aeb42df9ea19c3554b9 Parents: e96728c Author: Gopal V Authored: Tue Jul 31 13:48:07 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 13:53:03 2018 -0700 -- .../hive/ql/exec/vector/VectorGroupByOperator.java | 17 + 1 file changed, 9 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3bb4d1ca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java -- 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 75efc29..43f1162 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 @@ -725,10 +725,11 @@ public class VectorGroupByOperator extends Operator VectorHashKeyWrapper[] batchKeys = keyWrappersBatch.getVectorHashKeyWrappers(); + final VectorHashKeyWrapper prevKey = streamingKey; if (streamingKey == null) { // This is the first batch we process after switching from hash mode currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool(); -streamingKey = (VectorHashKeyWrapper) batchKeys[0].copyKey(); +streamingKey = batchKeys[0]; } aggregationBatchInfo.startBatch(); @@ -739,14 +740,9 @@ public class VectorGroupByOperator extends Operator // We've encountered a new key, must save current one // We can't forward yet, the aggregators have not been evaluated rowsToFlush[flushMark] = currentStreamingAggregators; - if (keysToFlush[flushMark] == null) { -keysToFlush[flushMark] = (VectorHashKeyWrapper) streamingKey.copyKey(); - } else { -streamingKey.duplicateTo(keysToFlush[flushMark]); - } - + keysToFlush[flushMark] = streamingKey; currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool(); - batchKeys[i].duplicateTo(streamingKey); + streamingKey = batchKeys[i]; ++flushMark; } aggregationBatchInfo.mapAggregationBufferSet(currentStreamingAggregators, i); @@ -759,8 +755,13 @@ public class VectorGroupByOperator extends Operator for (int i = 0; i < flushMark; ++i) { writeSingleRow(keysToFlush[i], rowsToFlush[i]); rowsToFlush[i].reset(); +keysToFlush[i] = null; streamAggregationBufferRowPool.putInPool(rowsToFlush[i]); } + + if (streamingKey != prevKey) { +streamingKey = (VectorHashKeyWrapper) streamingKey.copyKey(); + } } @Override
[2/3] hive git commit: HIVE-20040: JDBC: HTTP listen queue is 50 and SYNs are lost (Gopal V, reviewed by Prasanth Jayachandran)
HIVE-20040: JDBC: HTTP listen queue is 50 and SYNs are lost (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/80ae3332 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/80ae3332 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/80ae3332 Branch: refs/heads/master Commit: 80ae333272803945e5a44e34bb0370aac30357da Parents: 4d25151 Author: Gopal V Authored: Tue Jul 31 13:49:14 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 13:49:14 2018 -0700 -- .../org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java| 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/80ae3332/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java -- 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 95d78f8..89271d7 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 @@ -143,6 +143,7 @@ public class ThriftHttpCLIService extends ThriftCLIService { int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, TimeUnit.MILLISECONDS); connector.setIdleTimeout(maxIdleTime); + connector.setAcceptQueueSize(maxWorkerThreads); server.addConnector(connector);
[1/3] hive git commit: HIVE-20177: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode (Gopal V, reviewed by Matt McCline)
Repository: hive Updated Branches: refs/heads/master 3104d4756 -> 3e46515d3 HIVE-20177: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode (Gopal V, reviewed by Matt McCline) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4d251514 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4d251514 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4d251514 Branch: refs/heads/master Commit: 4d251514fde46a28f1d59d439097918576c26560 Parents: 3104d47 Author: Gopal V Authored: Tue Jul 31 13:48:07 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 13:48:07 2018 -0700 -- .../hive/ql/exec/vector/VectorGroupByOperator.java | 17 + 1 file changed, 9 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/4d251514/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java -- 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 75efc29..43f1162 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 @@ -725,10 +725,11 @@ public class VectorGroupByOperator extends Operator VectorHashKeyWrapper[] batchKeys = keyWrappersBatch.getVectorHashKeyWrappers(); + final VectorHashKeyWrapper prevKey = streamingKey; if (streamingKey == null) { // This is the first batch we process after switching from hash mode currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool(); -streamingKey = (VectorHashKeyWrapper) batchKeys[0].copyKey(); +streamingKey = batchKeys[0]; } aggregationBatchInfo.startBatch(); @@ -739,14 +740,9 @@ public class VectorGroupByOperator extends Operator // We've encountered a new key, must save current one // We can't forward yet, the aggregators have not been evaluated rowsToFlush[flushMark] = currentStreamingAggregators; - if (keysToFlush[flushMark] == null) { -keysToFlush[flushMark] = (VectorHashKeyWrapper) streamingKey.copyKey(); - } else { -streamingKey.duplicateTo(keysToFlush[flushMark]); - } - + keysToFlush[flushMark] = streamingKey; currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool(); - batchKeys[i].duplicateTo(streamingKey); + streamingKey = batchKeys[i]; ++flushMark; } aggregationBatchInfo.mapAggregationBufferSet(currentStreamingAggregators, i); @@ -759,8 +755,13 @@ public class VectorGroupByOperator extends Operator for (int i = 0; i < flushMark; ++i) { writeSingleRow(keysToFlush[i], rowsToFlush[i]); rowsToFlush[i].reset(); +keysToFlush[i] = null; streamAggregationBufferRowPool.putInPool(rowsToFlush[i]); } + + if (streamingKey != prevKey) { +streamingKey = (VectorHashKeyWrapper) streamingKey.copyKey(); + } } @Override
[3/3] hive git commit: HIVE-19199: ACID: DbTxnManager heartbeat-service needs static sync init (Gopal V, reviewed by Eugene Koifman)
HIVE-19199: ACID: DbTxnManager heartbeat-service needs static sync init (Gopal V, reviewed by Eugene Koifman) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3e46515d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3e46515d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3e46515d Branch: refs/heads/master Commit: 3e46515d3d556205cb643908565a24f6634c27c1 Parents: 80ae333 Author: Gopal V Authored: Tue Jul 31 13:51:53 2018 -0700 Committer: Gopal V Committed: Tue Jul 31 13:51:53 2018 -0700 -- .../hadoop/hive/ql/lockmgr/DbTxnManager.java| 31 +++- 1 file changed, 17 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3e46515d/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java -- 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 27abaf5..f746d2d 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 @@ -785,21 +785,24 @@ public final class DbTxnManager extends HiveTxnManagerImpl { } private synchronized void initHeartbeatExecutorService() { -if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown() -&& !heartbeatExecutorService.isTerminated()) { - return; +synchronized (DbTxnManager.class) { + if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown() + && !heartbeatExecutorService.isTerminated()) { +return; + } + heartbeatExecutorService = + Executors.newScheduledThreadPool( + conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE), + new ThreadFactory() { +private final AtomicInteger threadCounter = new AtomicInteger(); + +@Override +public Thread newThread(Runnable r) { + return new HeartbeaterThread(r, "Heartbeater-" + threadCounter.getAndIncrement()); +} + }); + ((ScheduledThreadPoolExecutor) heartbeatExecutorService).setRemoveOnCancelPolicy(true); } -heartbeatExecutorService = -Executors.newScheduledThreadPool( - conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE), new ThreadFactory() { - private final AtomicInteger threadCounter = new AtomicInteger(); - - @Override - public Thread newThread(Runnable r) { -return new HeartbeaterThread(r, "Heartbeater-" + threadCounter.getAndIncrement()); - } -}); -((ScheduledThreadPoolExecutor) heartbeatExecutorService).setRemoveOnCancelPolicy(true); } public static class HeartbeaterThread extends Thread {
hive git commit: HIVE-20165: Enable ZLIB for streaming ingest (Prasanth Jayachandran, via Gopal V)
Repository: hive Updated Branches: refs/heads/master 4fcf3d720 -> bac1d98c5 HIVE-20165: Enable ZLIB for streaming ingest (Prasanth Jayachandran, via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bac1d98c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bac1d98c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bac1d98c Branch: refs/heads/master Commit: bac1d98c5e91cdb39567f21b2068e7951a93ef44 Parents: 4fcf3d7 Author: Prasanth Jayachandran Authored: Tue Jul 17 20:17:55 2018 -0700 Committer: Gopal V Committed: Tue Jul 17 20:17:55 2018 -0700 -- .../hadoop/hive/ql/io/orc/OrcRecordUpdater.java | 1 - .../apache/hive/streaming/TestStreaming.java| 21 ++-- 2 files changed, 10 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/bac1d98c/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java -- 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 5590470..98f5df1 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 @@ -342,7 +342,6 @@ public class OrcRecordUpdater implements RecordUpdater { writerOptions.blockPadding(false); if (optionsCloneForDelta.getConfiguration().getBoolean( HiveConf.ConfVars.HIVE_ORC_DELTA_STREAMING_OPTIMIZATIONS_ENABLED.varname, false)) { -writerOptions.compress(CompressionKind.NONE); writerOptions.encodingStrategy(org.apache.orc.OrcFile.EncodingStrategy.SPEED); writerOptions.rowIndexStride(0); writerOptions.getConfiguration().set(OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getAttribute(), "-1.0"); http://git-wip-us.apache.org/repos/asf/hive/blob/bac1d98c/streaming/src/test/org/apache/hive/streaming/TestStreaming.java -- diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java index 1f05d88..8b5e508 100644 --- a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java +++ b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java @@ -2072,20 +2072,19 @@ public class TestStreaming { System.setOut(origOut); String outDump = new String(myOut.toByteArray()); -// make sure delta files are written with no indexes, no compression and no dictionary -// no compression -Assert.assertEquals(true, outDump.contains("Compression: NONE")); +// make sure delta files are written with no indexes and no dictionary +Assert.assertEquals(true, outDump.contains("Compression: ZLIB")); // no stats/indexes Assert.assertEquals(true, outDump.contains("Column 0: count: 0 hasNull: false")); -Assert.assertEquals(true, outDump.contains("Column 1: count: 0 hasNull: false bytesOnDisk: 12 sum: 0")); -Assert.assertEquals(true, outDump.contains("Column 2: count: 0 hasNull: false bytesOnDisk: 12 sum: 0")); -Assert.assertEquals(true, outDump.contains("Column 3: count: 0 hasNull: false bytesOnDisk: 24 sum: 0")); -Assert.assertEquals(true, outDump.contains("Column 4: count: 0 hasNull: false bytesOnDisk: 14 sum: 0")); -Assert.assertEquals(true, outDump.contains("Column 5: count: 0 hasNull: false bytesOnDisk: 12 sum: 0")); +Assert.assertEquals(true, outDump.contains("Column 1: count: 0 hasNull: false bytesOnDisk: 15 sum: 0")); +Assert.assertEquals(true, outDump.contains("Column 2: count: 0 hasNull: false bytesOnDisk: 15 sum: 0")); +Assert.assertEquals(true, outDump.contains("Column 3: count: 0 hasNull: false bytesOnDisk: 19 sum: 0")); +Assert.assertEquals(true, outDump.contains("Column 4: count: 0 hasNull: false bytesOnDisk: 17 sum: 0")); +Assert.assertEquals(true, outDump.contains("Column 5: count: 0 hasNull: false bytesOnDisk: 15 sum: 0")); Assert.assertEquals(true, outDump.contains("Column 6: count: 0 hasNull: false")); -Assert.assertEquals(true, outDump.contains("Column 7: count: 0 hasNull: false bytesOnDisk: 11864")); -Assert.assertEquals(true, outDump.contains("Column 8: count: 0 hasNull: false bytesOnDisk: 2033 sum: 0")); -Assert.assertEquals(true, outDump.contains("Column 9: count: 0 hasNull: false bytesOnDisk: 13629")); +Assert.assertEquals(true, outDump.contains("Column 7: count: 0 hasNull: false bytesOnDisk: 3929")); +Assert.assertEquals(true, outDump.contains("Column 8: count: 0 hasNull: false bytesOnDisk: 1484 sum: 0")); +Assert.assertEquals(true, outDump.contains("Column 9: count:
hive git commit: HIVE-20041: ResultsCache: Improve logging for concurrent queries (Laszlo Bodor reviewed by Gopal V)
Repository: hive Updated Branches: refs/heads/master 20eb7b516 -> d8306cf68 HIVE-20041: ResultsCache: Improve logging for concurrent queries (Laszlo Bodor reviewed by Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d8306cf6 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d8306cf6 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d8306cf6 Branch: refs/heads/master Commit: d8306cf68087ed369178c6f3a343db07ad7b9ca8 Parents: 20eb7b5 Author: Laszlo Bodor Authored: Thu Jul 12 23:43:05 2018 -0700 Committer: Gopal V Committed: Thu Jul 12 23:43:05 2018 -0700 -- ql/src/java/org/apache/hadoop/hive/ql/Driver.java| 5 +++-- .../hadoop/hive/ql/cache/results/QueryResultsCache.java | 8 2 files changed, 7 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d8306cf6/ql/src/java/org/apache/hadoop/hive/ql/Driver.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index bf4d29c..78922f1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -2203,10 +2203,11 @@ public class Driver implements IDriver { if (plan.hasAcidResourcesInQuery()) { txnWriteIdList = AcidUtils.getValidTxnWriteIdList(conf); } +CacheEntry cacheEntry = cacheUsage.getCacheEntry(); boolean savedToCache = QueryResultsCache.getInstance().setEntryValid( -cacheUsage.getCacheEntry(), +cacheEntry, plan.getFetchTask().getWork()); -LOG.info("savedToCache: {}", savedToCache); +LOG.info("savedToCache: {} ({})", savedToCache, cacheEntry); if (savedToCache) { useFetchFromCache(cacheUsage.getCacheEntry()); // setEntryValid() already increments the reader count. Set usedCacheEntry so it gets released. http://git-wip-us.apache.org/repos/asf/hive/blob/d8306cf6/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java b/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java index d29c4da..44f7bf8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java @@ -213,9 +213,9 @@ public final class QueryResultsCache { } public String toString() { - return "CacheEntry query: [" + getQueryInfo().getLookupInfo().getQueryText() - + "], status: " + status + ", location: " + cachedResultsPath - + ", size: " + size; + return String.format("CacheEntry#%s query: [ %s ], status: %s, location: %s, size: %d", + System.identityHashCode(this), getQueryInfo().getLookupInfo().getQueryText(), status, + cachedResultsPath, size); } public boolean addReader() { @@ -297,7 +297,7 @@ public final class QueryResultsCache { * false if the status changes from PENDING to INVALID */ public boolean waitForValidStatus() { - LOG.info("Waiting on pending cacheEntry"); + LOG.info("Waiting on pending cacheEntry: {}", this); long timeout = 1000; long startTime = System.nanoTime();
hive git commit: HIVE-20099: Fix logger for LlapServlet (Rajkumar Singh via Gopal V)
Repository: hive Updated Branches: refs/heads/branch-3 369798d90 -> fc2c22017 HIVE-20099: Fix logger for LlapServlet (Rajkumar Singh via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fc2c2201 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fc2c2201 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fc2c2201 Branch: refs/heads/branch-3 Commit: fc2c2201701ef5dfdcd7d6586419207b593d8fe6 Parents: 369798d Author: Raj Kumar Singh Authored: Mon Jul 9 09:19:07 2018 -0700 Committer: Gopal V Committed: Mon Jul 9 09:38:31 2018 -0700 -- service/src/java/org/apache/hive/http/LlapServlet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fc2c2201/service/src/java/org/apache/hive/http/LlapServlet.java -- diff --git a/service/src/java/org/apache/hive/http/LlapServlet.java b/service/src/java/org/apache/hive/http/LlapServlet.java index a0eb409..2bc59b5 100644 --- a/service/src/java/org/apache/hive/http/LlapServlet.java +++ b/service/src/java/org/apache/hive/http/LlapServlet.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hive.llap.cli.LlapStatusServiceDriver; @SuppressWarnings("serial") public class LlapServlet extends HttpServlet { - private static final Log LOG = LogFactory.getLog(JMXJsonServlet.class); + private static final Log LOG = LogFactory.getLog(LlapServlet.class); /** * Initialize this servlet.
[2/3] hive git commit: HIVE-20099: Fix logger for LlapServlet (Rajkumar Singh via Gopal V)
HIVE-20099: Fix logger for LlapServlet (Rajkumar Singh via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/eb91fa49 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/eb91fa49 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/eb91fa49 Branch: refs/heads/master Commit: eb91fa49db99396c3cf0e79d02962dbbaef55e06 Parents: d3bba71 Author: Raj Kumar Singh Authored: Mon Jul 9 09:19:07 2018 -0700 Committer: Gopal V Committed: Mon Jul 9 09:19:07 2018 -0700 -- service/src/java/org/apache/hive/http/LlapServlet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/eb91fa49/service/src/java/org/apache/hive/http/LlapServlet.java -- diff --git a/service/src/java/org/apache/hive/http/LlapServlet.java b/service/src/java/org/apache/hive/http/LlapServlet.java index a0eb409..2bc59b5 100644 --- a/service/src/java/org/apache/hive/http/LlapServlet.java +++ b/service/src/java/org/apache/hive/http/LlapServlet.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hive.llap.cli.LlapStatusServiceDriver; @SuppressWarnings("serial") public class LlapServlet extends HttpServlet { - private static final Log LOG = LogFactory.getLog(JMXJsonServlet.class); + private static final Log LOG = LogFactory.getLog(LlapServlet.class); /** * Initialize this servlet.
[1/3] hive git commit: HIVE-20103: WM: Only Aggregate DAG counters if at least one is used (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master 9fb42927a -> db1a70eb3 HIVE-20103: WM: Only Aggregate DAG counters if at least one is used (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d3bba716 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d3bba716 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d3bba716 Branch: refs/heads/master Commit: d3bba71666c41f13e8efb007d7cd20accde0b296 Parents: 9fb4292 Author: Gopal V Authored: Mon Jul 9 09:08:21 2018 -0700 Committer: Gopal V Committed: Mon Jul 9 09:08:21 2018 -0700 -- .../ql/exec/tez/monitoring/TezJobMonitor.java| 19 ++- 1 file changed, 14 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d3bba716/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java -- 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 98048c8..d3fe190 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 @@ -166,17 +166,26 @@ public class TezJobMonitor { context.checkHeartbeaterLockException(); } -status = dagClient.getDAGStatus(EnumSet.of(StatusGetOpts.GET_COUNTERS), checkInterval); -TezCounters dagCounters = status.getDAGCounters(); -vertexProgressMap = status.getVertexProgress(); wmContext = context.getWmContext(); +EnumSet opts = null; +if (wmContext != null) { + Set desiredCounters = wmContext.getSubscribedCounters(); + if (desiredCounters != null && !desiredCounters.isEmpty()) { +opts = EnumSet.of(StatusGetOpts.GET_COUNTERS); + } +} + +status = dagClient.getDAGStatus(opts, checkInterval); + +vertexProgressMap = status.getVertexProgress(); List vertexNames = vertexProgressMap.keySet() .stream() .map(k -> k.replaceAll(" ", "_")) .collect(Collectors.toList()); -if (dagCounters != null && wmContext != null) { +if (wmContext != null) { Set desiredCounters = wmContext.getSubscribedCounters(); - if (desiredCounters != null && !desiredCounters.isEmpty()) { + TezCounters dagCounters = status.getDAGCounters(); + if (dagCounters != null && desiredCounters != null && !desiredCounters.isEmpty()) { Map currentCounters = getCounterValues(dagCounters, vertexNames, vertexProgressMap, desiredCounters, done); if (LOG.isDebugEnabled()) {
[3/3] hive git commit: HIVE-20043: HiveServer2: SessionState has a static sync block around an AtomicBoolean (Laszlo Bodor, reviewed by Zoltan Haindrich)
HIVE-20043: HiveServer2: SessionState has a static sync block around an AtomicBoolean (Laszlo Bodor, reviewed by Zoltan Haindrich) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/db1a70eb Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/db1a70eb Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/db1a70eb Branch: refs/heads/master Commit: db1a70eb361e56f57bd10fcc0cbf21b99866fcdf Parents: eb91fa4 Author: Laszlo Bodor Authored: Mon Jul 9 09:31:43 2018 -0700 Committer: Gopal V Committed: Mon Jul 9 09:32:18 2018 -0700 -- .../java/org/apache/hadoop/hive/ql/session/SessionState.java | 6 ++ 1 file changed, 2 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/db1a70eb/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java -- 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 27f0216..262bbb9 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 @@ -607,10 +607,8 @@ public class SessionState { private static void start(SessionState startSs, boolean isAsync, LogHelper console) { setCurrentSessionState(startSs); -synchronized(SessionState.class) { - if (!startSs.isStarted.compareAndSet(false, true)) { -return; - } +if (!startSs.isStarted.compareAndSet(false, true)) { + return; } if (startSs.hiveHist == null){
hive git commit: HIVE-20021: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master 1cedb1209 -> 1fc122153 HIVE-20021: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1fc12215 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1fc12215 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1fc12215 Branch: refs/heads/master Commit: 1fc1221536671f28ac28a3e4d99969a704cf740a Parents: 1cedb12 Author: Gopal V Authored: Mon Jul 2 15:54:24 2018 -0700 Committer: Gopal V Committed: Mon Jul 2 15:56:08 2018 -0700 -- .../src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1fc12215/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java -- 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 5a77122..02490f1 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 @@ -739,6 +739,9 @@ public class Hadoop23Shims extends HadoopShimsSecure { @Override public Long getFileId() { + if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) { +return null; + } return fileId; } }
hive git commit: HIVE-20021: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/branch-3 f5bbf4d0e -> 6ddfa8232 HIVE-20021: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6ddfa823 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6ddfa823 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6ddfa823 Branch: refs/heads/branch-3 Commit: 6ddfa8232f77af0b2c7638cf0d6cfc1dc22f5bc9 Parents: f5bbf4d Author: Gopal V Authored: Mon Jul 2 15:54:24 2018 -0700 Committer: Gopal V Committed: Mon Jul 2 15:54:24 2018 -0700 -- .../src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/6ddfa823/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java -- 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 ec06a88..79c69ee 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 @@ -737,6 +737,9 @@ public class Hadoop23Shims extends HadoopShimsSecure { @Override public Long getFileId() { + if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) { +return null; + } return fileId; } }
hive git commit: HIVE-19890: ACID: Inherit bucket-id from original ROW_ID for delete deltas (Gopal V, reviewed by Eugene Koifman)
Repository: hive Updated Branches: refs/heads/branch-3 5b47ee6cc -> 5f2303c2c HIVE-19890: ACID: Inherit bucket-id from original ROW_ID for delete deltas (Gopal V, reviewed by Eugene Koifman) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5f2303c2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5f2303c2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5f2303c2 Branch: refs/heads/branch-3 Commit: 5f2303c2c97fd51ab30e4db4dd7bbfc33414ddec Parents: 5b47ee6 Author: Gopal V Authored: Mon Jun 25 12:55:24 2018 -0700 Committer: Gopal V Committed: Mon Jun 25 12:55:40 2018 -0700 -- .../hadoop/hive/ql/exec/FileSinkOperator.java | 58 .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 6 ++ .../apache/hadoop/hive/ql/TestTxnNoBuckets.java | 34 .../clientpositive/llap/acid_no_buckets.q.out | 6 +- .../materialized_view_create_rewrite_4.q.out| 2 + 5 files changed, 73 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/5f2303c2/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java -- 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 9c57eff..2a74f86 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 @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.Serializable; import java.io.StringWriter; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -378,6 +379,38 @@ public class FileSinkOperator extends TerminalOperator implements public Collection getStoredStats() { return stat.getStoredStats(); } + +/** + * This method is intended for use with ACID unbucketed tables, where the DELETE ops behave as + * though they are bucketed, but without an explicit pre-specified bucket count. The bucketNum + * is read out of the middle value of the ROW__ID variable and this is written out from a single + * FileSink, in ways similar to the multi file spray, but without knowing the total number of + * buckets ahead of time. + * + * ROW__ID (1,2[0],3) => bucket_2 + * ROW__ID (1,3[0],4) => bucket_3 etc + * + * A new FSP is created for each partition, so this only requires the bucket numbering and that + * is mapped in directly as an index. + */ +public int createDynamicBucket(int bucketNum) { + // this assumes all paths are bucket names (which means no lookup is needed) + int writerOffset = bucketNum; + if (updaters.length <= writerOffset) { +this.updaters = Arrays.copyOf(updaters, writerOffset + 1); +this.outPaths = Arrays.copyOf(outPaths, writerOffset + 1); +this.finalPaths = Arrays.copyOf(finalPaths, writerOffset + 1); + } + + if (this.finalPaths[writerOffset] == null) { +// uninitialized bucket +String bucketName = +Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), bucketNum); +this.finalPaths[writerOffset] = new Path(bDynParts ? buildTmpPath() : parent, bucketName); +this.outPaths[writerOffset] = new Path(buildTaskOutputTempPath(), bucketName); + } + return writerOffset; +} } // class FSPaths private static final long serialVersionUID = 1L; @@ -978,31 +1011,12 @@ public class FileSinkOperator extends TerminalOperator implements " from data but no mapping in 'bucketMap'." + extraMsg); } writerOffset = bucketMap.get(bucketNum); +} else if (!isBucketed) { + writerOffset = fpaths.createDynamicBucket(bucketNum); } if (fpaths.updaters[writerOffset] == null) { - /*data for delete commands always have ROW__ID which implies that the bucket ID - * for each row is known. RecordUpdater creates bucket_N file based on 'bucketNum' thus - * delete events always land in the proper bucket_N file. This could even handle - * cases where multiple writers are writing bucket_N file for the same N in which case - * Hive.copyFiles() will make one of them bucket_N_copy_M in the final location. The - * reset of acid (read path) doesn't know how to handle copy_N files except for 'original' - * files (HIVE-16177)*/ - int writerId = -1; - if(!isBucketed) { -assert !multiFileSpray; -assert writerOffset == 0; -/**For un-bucketed tables, Deletes with ROW__IDs w
hive git commit: HIVE-19890: ACID: Inherit bucket-id from original ROW_ID for delete deltas (Gopal V, reviewed by Eugene Koifman)
Repository: hive Updated Branches: refs/heads/master 6d532e7c4 -> 23d2b80b0 HIVE-19890: ACID: Inherit bucket-id from original ROW_ID for delete deltas (Gopal V, reviewed by Eugene Koifman) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/23d2b80b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/23d2b80b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/23d2b80b Branch: refs/heads/master Commit: 23d2b80b0ae246d00613b06ce5ed554efb49d1d4 Parents: 6d532e7 Author: Gopal V Authored: Fri Jun 22 15:21:10 2018 -0700 Committer: Gopal V Committed: Fri Jun 22 15:24:00 2018 -0700 -- .../hadoop/hive/ql/exec/FileSinkOperator.java | 58 .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 6 ++ .../apache/hadoop/hive/ql/TestTxnNoBuckets.java | 34 .../clientpositive/llap/acid_no_buckets.q.out | 6 +- .../materialized_view_create_rewrite_4.q.out| 2 + 5 files changed, 73 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/23d2b80b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java -- 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 c2319bb..21f8268 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 @@ -23,6 +23,7 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TEMPORARY_TABLE import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -370,6 +371,38 @@ public class FileSinkOperator extends TerminalOperator implements public Collection getStoredStats() { return stat.getStoredStats(); } + +/** + * This method is intended for use with ACID unbucketed tables, where the DELETE ops behave as + * though they are bucketed, but without an explicit pre-specified bucket count. The bucketNum + * is read out of the middle value of the ROW__ID variable and this is written out from a single + * FileSink, in ways similar to the multi file spray, but without knowing the total number of + * buckets ahead of time. + * + * ROW__ID (1,2[0],3) => bucket_2 + * ROW__ID (1,3[0],4) => bucket_3 etc + * + * A new FSP is created for each partition, so this only requires the bucket numbering and that + * is mapped in directly as an index. + */ +public int createDynamicBucket(int bucketNum) { + // this assumes all paths are bucket names (which means no lookup is needed) + int writerOffset = bucketNum; + if (updaters.length <= writerOffset) { +this.updaters = Arrays.copyOf(updaters, writerOffset + 1); +this.outPaths = Arrays.copyOf(outPaths, writerOffset + 1); +this.finalPaths = Arrays.copyOf(finalPaths, writerOffset + 1); + } + + if (this.finalPaths[writerOffset] == null) { +// uninitialized bucket +String bucketName = +Utilities.replaceTaskIdFromFilename(Utilities.getTaskId(hconf), bucketNum); +this.finalPaths[writerOffset] = new Path(bDynParts ? buildTmpPath() : parent, bucketName); +this.outPaths[writerOffset] = new Path(buildTaskOutputTempPath(), bucketName); + } + return writerOffset; +} } // class FSPaths private static final long serialVersionUID = 1L; @@ -976,31 +1009,12 @@ public class FileSinkOperator extends TerminalOperator implements " from data but no mapping in 'bucketMap'." + extraMsg); } writerOffset = bucketMap.get(bucketNum); +} else if (!isBucketed) { + writerOffset = fpaths.createDynamicBucket(bucketNum); } if (fpaths.updaters[writerOffset] == null) { - /*data for delete commands always have ROW__ID which implies that the bucket ID - * for each row is known. RecordUpdater creates bucket_N file based on 'bucketNum' thus - * delete events always land in the proper bucket_N file. This could even handle - * cases where multiple writers are writing bucket_N file for the same N in which case - * Hive.copyFiles() will make one of them bucket_N_copy_M in the final location. The - * reset of acid (read path) doesn't know how to handle copy_N files except for 'original' - * files (HIVE-16177)*/ - int writerId = -1; - if(!isBucketed) { -assert !multiFileSpray; -assert writerOffset == 0; -/
hive git commit: HIVE-19038: LLAP: Service loader throws Provider not found exception if hive-llap-server is in class path while loading tokens (Arun Mahadevan, via Gopal Vijayaraghavan)
Repository: hive Updated Branches: refs/heads/branch-3 559ebb3e9 -> 473d1b936 HIVE-19038: LLAP: Service loader throws Provider not found exception if hive-llap-server is in class path while loading tokens (Arun Mahadevan, via Gopal Vijayaraghavan) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/473d1b93 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/473d1b93 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/473d1b93 Branch: refs/heads/branch-3 Commit: 473d1b936fda0d5d94dea81f45ab80e37335775d Parents: 559ebb3 Author: Arun Mahadevan Authored: Mon Apr 16 10:08:16 2018 -0700 Committer: Gopal V Committed: Mon Apr 16 13:34:26 2018 -0700 -- .../services/org.apache.hadoop.security.token.TokenRenewer | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/473d1b93/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer -- diff --git a/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer b/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer index e80ac41..c3215a9 100644 --- a/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer +++ b/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer @@ -11,4 +11,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -org.apache.hadoop.hive.llap.security.LlapTokenIdentifier.Renewer +org.apache.hadoop.hive.llap.security.LlapTokenIdentifier$Renewer
hive git commit: HIVE-19038: LLAP: Service loader throws Provider not found exception if hive-llap-server is in class path while loading tokens (Arun Mahadevan, via Gopal Vijayaraghavan)
Repository: hive Updated Branches: refs/heads/master 996db32ba -> 1e9f3d77c HIVE-19038: LLAP: Service loader throws Provider not found exception if hive-llap-server is in class path while loading tokens (Arun Mahadevan, via Gopal Vijayaraghavan) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1e9f3d77 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1e9f3d77 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1e9f3d77 Branch: refs/heads/master Commit: 1e9f3d77cf814870ce5160904a5cee58a634e3b9 Parents: 996db32 Author: Arun Mahadevan Authored: Mon Apr 16 10:08:16 2018 -0700 Committer: Gopal V Committed: Mon Apr 16 10:08:16 2018 -0700 -- .../services/org.apache.hadoop.security.token.TokenRenewer | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1e9f3d77/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer -- diff --git a/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer b/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer index e80ac41..c3215a9 100644 --- a/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer +++ b/llap-server/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer @@ -11,4 +11,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -org.apache.hadoop.hive.llap.security.LlapTokenIdentifier.Renewer +org.apache.hadoop.hive.llap.security.LlapTokenIdentifier$Renewer
hive git commit: HIVE-19085: FastHiveDecimal abs(0) sets sign to +ve (Gopal V, reviewed by Matt McCline)
Repository: hive Updated Branches: refs/heads/master 105cc6543 -> 2f802e908 HIVE-19085: FastHiveDecimal abs(0) sets sign to +ve (Gopal V, reviewed by Matt McCline) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2f802e90 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2f802e90 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2f802e90 Branch: refs/heads/master Commit: 2f802e908fd1e69ab2d1885b994faa64c290687e Parents: 105cc65 Author: Gopal V Authored: Thu Apr 5 17:28:30 2018 -0700 Committer: Gopal V Committed: Thu Apr 5 17:28:54 2018 -0700 -- .../java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/2f802e90/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java -- diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java index de9149b..7151f09 100644 --- a/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java +++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java @@ -428,6 +428,9 @@ public class FastHiveDecimal { } protected void fastAbs() { +if (fastSignum == 0) { + return; +} fastSignum = 1; }
hive git commit: HIVE-18886: ACID: NPE on unexplained mysql exceptions (Gopal V, reviewed by Eugene Koifman)
Repository: hive Updated Branches: refs/heads/master e7480d79a -> d2d50e694 HIVE-18886: ACID: NPE on unexplained mysql exceptions (Gopal V, reviewed by Eugene Koifman) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d2d50e69 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d2d50e69 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d2d50e69 Branch: refs/heads/master Commit: d2d50e6940438f9cb1309aea9d6f278ed93b0536 Parents: e7480d7 Author: Gopal V Authored: Fri Mar 16 16:54:49 2018 -0700 Committer: Gopal V Committed: Fri Mar 16 16:54:49 2018 -0700 -- .../org/apache/hadoop/hive/metastore/DatabaseProduct.java| 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d2d50e69/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java -- diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java index 03a5e4a..0b3504d 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java +++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java @@ -53,10 +53,10 @@ public enum DatabaseProduct { public static boolean isDeadlock(DatabaseProduct dbProduct, SQLException e) { return e instanceof SQLTransactionRollbackException || ((dbProduct == MYSQL || dbProduct == POSTGRES || dbProduct == SQLSERVER) -&& e.getSQLState().equals("40001")) -|| (dbProduct == POSTGRES && e.getSQLState().equals("40P01")) -|| (dbProduct == ORACLE && (e.getMessage().contains("deadlock detected") -|| e.getMessage().contains("can't serialize access for this transaction"))); +&& "40001".equals(e.getSQLState())) +|| (dbProduct == POSTGRES && "40P01".equals(e.getSQLState())) +|| (dbProduct == ORACLE && (e.getMessage() != null && (e.getMessage().contains("deadlock detected") +|| e.getMessage().contains("can't serialize access for this transaction"; } /**
hive git commit: HIVE-18610: Performance: ListKeyWrapper does not check for hashcode equals, before comparing members (Gopal V, reviewed by Ashutosh Chauhan)
Repository: hive Updated Branches: refs/heads/master abede8ee2 -> cd8eda856 HIVE-18610: Performance: ListKeyWrapper does not check for hashcode equals, before comparing members (Gopal V, reviewed by Ashutosh Chauhan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cd8eda85 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cd8eda85 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cd8eda85 Branch: refs/heads/master Commit: cd8eda85676341e697720c90550b61360266159d Parents: abede8e Author: Gopal V Authored: Tue Mar 6 22:08:23 2018 -0800 Committer: Gopal V Committed: Tue Mar 6 22:08:39 2018 -0800 -- .../hadoop/hive/ql/exec/KeyWrapperFactory.java | 17 ++--- .../hadoop/hive/ql/exec/TestKeyWrapperFactory.java | 8 .../objectinspector/ObjectInspectorUtils.java | 15 ++- 3 files changed, 28 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/cd8eda85/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java index 73683ff..3c7f0b7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java @@ -18,10 +18,7 @@ package org.apache.hadoop.hive.ql.exec; -import java.util.Arrays; - import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.serde2.lazy.LazyDouble; import org.apache.hadoop.hive.serde2.objectinspector.ListObjectsEqualComparer; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; @@ -29,10 +26,10 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import org.apache.hadoop.io.DoubleWritable; import org.apache.hadoop.io.Text; public class KeyWrapperFactory { + public KeyWrapperFactory(ExprNodeEvaluator[] keyFields, ObjectInspector[] keyObjectInspectors, ObjectInspector[] currentKeyObjectInspectors) { this.keyFields = keyFields; @@ -66,7 +63,7 @@ public class KeyWrapperFactory { transient ListObjectsEqualComparer newKeyStructEqualComparer; class ListKeyWrapper extends KeyWrapper { -int hashcode; +int hashcode = -1; Object[] keys; // decide whether this is already in hashmap (keys in hashmap are deepcopied // version, and we need to use 'currentKeyObjectInspector'). @@ -102,8 +99,13 @@ public class KeyWrapperFactory { if (!(obj instanceof ListKeyWrapper)) { return false; } - Object[] copied_in_hashmap = ((ListKeyWrapper) obj).keys; - return equalComparer.areEqual(copied_in_hashmap, keys); + ListKeyWrapper other = ((ListKeyWrapper) obj); + if (other.hashcode != this.hashcode && this.hashcode != -1 && other.hashcode != -1) { +return false; + } + Object[] copied_in_hashmap = other.keys; + boolean result = equalComparer.areEqual(copied_in_hashmap, keys); + return result; } @Override @@ -117,6 +119,7 @@ public class KeyWrapperFactory { for (int i = 0; i < keyFields.length; i++) { keys[i] = keyFields[i].evaluate(row); } + hashcode = -1; } @Override http://git-wip-us.apache.org/repos/asf/hive/blob/cd8eda85/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java index 4de6dd0..03c4ed6 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java @@ -110,4 +110,12 @@ public class TestKeyWrapperFactory { assertFalse(w3.equals(w4)); assertFalse(w4.equals(w3)); } + + @Test + public void testUnsetHashCode() { +KeyWrapper w1 = factory.getKeyWrapper(); +KeyWrapper w2 = w1.copyKey(); +w1.setHashKey(); +assertTrue(w1.equals(w2)); + } } http://git-wip-us.apache.org/repos/asf/hive/blob/cd8eda85/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java -- diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/obje
hive git commit: HIVE-18713: addendum; fix q.out mid-air conflict
Repository: hive Updated Branches: refs/heads/master 2006fb8d2 -> 7a8f10599 HIVE-18713: addendum; fix q.out mid-air conflict Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7a8f1059 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7a8f1059 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7a8f1059 Branch: refs/heads/master Commit: 7a8f105999fb6bc2dc9e444588d78f848002dca4 Parents: 2006fb8 Author: Gopal V Authored: Thu Feb 22 13:39:56 2018 -0800 Committer: Gopal V Committed: Thu Feb 22 13:40:09 2018 -0800 -- .../llap/vectorized_mapjoin3.q.out | 100 --- 1 file changed, 65 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7a8f1059/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out -- diff --git a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out index 4fb3466..00286ce 100644 --- a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out +++ b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out @@ -43,7 +43,6 @@ POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@table_6 POSTHOOK: Lineage: table_6.int_col_0 SCRIPT [] -Warning: Map Join MAPJOIN[16][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: SELECT t1.decimal0801_col FROM table_19 t1 WHERE (SELECT max(tt1.int_col_0) AS int_col FROM table_6 tt1) IN (t1.int_col_1) AND decimal0801_col is not null @@ -89,7 +88,6 @@ POSTHOOK: Input: default@table_19 POSTHOOK: Input: default@table_6 A masked pattern was here 418.9 -Warning: Map Join MAPJOIN[16][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: EXPLAIN VECTORIZATION DETAIL SELECT t1.decimal0801_col FROM table_19 t1 @@ -122,45 +120,69 @@ STAGE PLANS: TableScan alias: t1 Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE + TableScan Vectorization: + native: true + vectorizationSchemaColumns: [0:decimal0801_col:decimal(8,1), 1:int_col_1:int, 2:ROW__ID:struct] Filter Operator -predicate: decimal0801_col is not null (type: boolean) -Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE +Filter Vectorization: +className: VectorFilterOperator +native: true +predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 0:decimal(8,1)), SelectColumnIsNotNull(col 1:int)) +predicate: (decimal0801_col is not null and int_col_1 is not null) (type: boolean) +Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: decimal0801_col (type: decimal(8,1)), int_col_1 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Map Join Operator condition map: - Left Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 - 1 -outputColumnNames: _col0, _col1, _col2 + 0 _col1 (type: int) + 1 _col0 (type: int) +Map Join Vectorization: +bigTableKeyColumnNums: [1] +bigTableRetainedColumnNums: [0] +bigTableValueColumnNums: [0] +className: VectorMapJoinInnerBigOnlyLongOperator +native: true +nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true +projectedOutputColumnNums:
hive git commit: HIVE-18713: Optimize: Transform IN clauses to = when there's only one element (Gopal V, reviewed by Vineet Garg)
Repository: hive Updated Branches: refs/heads/master 0b03b819f -> 3f2838388 HIVE-18713: Optimize: Transform IN clauses to = when there's only one element (Gopal V, reviewed by Vineet Garg) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3f283838 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3f283838 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3f283838 Branch: refs/heads/master Commit: 3f2838388d67bcdb67c446cea30da227ba88c8bb Parents: 0b03b81 Author: Gopal V Authored: Wed Feb 21 09:27:22 2018 -0800 Committer: Gopal V Committed: Wed Feb 21 09:27:22 2018 -0800 -- .../calcite/translator/RexNodeConverter.java| 10 + .../clientpositive/cbo_rp_simple_select.q | 19 +- .../queries/clientpositive/cbo_simple_select.q | 18 ++ .../clientpositive/cbo_rp_simple_select.q.out | 234 +++ .../clientpositive/cbo_simple_select.q.out | 234 +++ .../clientpositive/llap/cbo_simple_select.q.out | 216 + .../clientpositive/perf/spark/query33.q.out | 2 +- .../clientpositive/perf/spark/query60.q.out | 2 +- .../clientpositive/perf/spark/query89.q.out | 8 +- .../clientpositive/perf/tez/query33.q.out | 2 +- .../clientpositive/perf/tez/query60.q.out | 2 +- .../clientpositive/perf/tez/query89.q.out | 6 +- .../spark/cbo_simple_select.q.out | 216 + .../clientpositive/vectorized_mapjoin3.q.out| 151 +--- 14 files changed, 1013 insertions(+), 107 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3f283838/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java -- 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 39ff591..7a482d9 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 @@ -257,6 +257,7 @@ public class RexNodeConverter { func.getChildren().size() != 0; boolean isBetween = !isNumeric && tgtUdf instanceof GenericUDFBetween; boolean isIN = !isNumeric && tgtUdf instanceof GenericUDFIn; +boolean isAllPrimitive = true; if (isNumeric) { tgtDT = func.getTypeInfo(); @@ -312,6 +313,8 @@ public class RexNodeConverter { } } + isAllPrimitive = + isAllPrimitive && tmpExprNode.getTypeInfo().getCategory() == Category.PRIMITIVE; argTypeBldr.add(TypeConverter.convert(tmpExprNode.getTypeInfo(), cluster.getTypeFactory())); tmpRN = convert(tmpExprNode); childRexNodeLst.add(tmpRN); @@ -336,6 +339,13 @@ public class RexNodeConverter { } else if (HiveFloorDate.ALL_FUNCTIONS.contains(calciteOp)) { // If it is a floor operator, we need to rewrite it childRexNodeLst = rewriteFloorDateChildren(calciteOp, childRexNodeLst); + } else if (calciteOp.getKind() == SqlKind.IN && childRexNodeLst.size() == 2 && isAllPrimitive) { +// if it is a single item in an IN clause, transform A IN (B) to A = B +// from IN [A,B] => EQUALS [A,B] +// except complex types +calciteOp = +SqlFunctionConverter.getCalciteOperator("=", FunctionRegistry.getFunctionInfo("=") +.getGenericUDF(), argTypeBldr.build(), retType); } expr = cluster.getRexBuilder().makeCall(retType, calciteOp, childRexNodeLst); } else { http://git-wip-us.apache.org/repos/asf/hive/blob/3f283838/ql/src/test/queries/clientpositive/cbo_rp_simple_select.q -- diff --git a/ql/src/test/queries/clientpositive/cbo_rp_simple_select.q b/ql/src/test/queries/clientpositive/cbo_rp_simple_select.q index 6be2b19..deb0f8f 100644 --- a/ql/src/test/queries/clientpositive/cbo_rp_simple_select.q +++ b/ql/src/test/queries/clientpositive/cbo_rp_simple_select.q @@ -45,8 +45,6 @@ select * from (select cbo_t2.key as x, c_int as c_int, (((c_int+c_float)*10)+5) select cbo_t1.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t1 where cbo_t1.c_int >= 0; select cbo_t2.c_int+c_float as x , c_int as c_int, (((c_int+c_float)*10)+5) as y from (select * from cbo_t1 where cbo_t1.c_int >= 0) as cbo_t2 where cbo_t2.c_int >= 0; - - -- 13. null expr in select list select null from cbo_t3; @@ -55,3 +53,20 @@ select key from cbo_t1 where c_int = -6 or c_int = +6; -- 15. query referencing only part
hive git commit: HIVE-18708: Vectorization: Delay out-of-tree fixups till whole work is vectorized (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master 545550071 -> 273a08ed4 HIVE-18708: Vectorization: Delay out-of-tree fixups till whole work is vectorized (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/273a08ed Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/273a08ed Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/273a08ed Branch: refs/heads/master Commit: 273a08ed4e705d9529da7d04e26d2f81bae6ecdb Parents: 5455500 Author: Gopal V Authored: Fri Feb 16 12:28:54 2018 -0800 Committer: Gopal V Committed: Fri Feb 16 12:28:54 2018 -0800 -- .../test/resources/testconfiguration.properties | 1 + .../hive/ql/optimizer/physical/Vectorizer.java | 39 +- .../clientpositive/vectorized_mapjoin3.q| 53 ++ .../llap/vectorized_mapjoin3.q.out | 699 +++ .../clientpositive/vectorized_mapjoin3.q.out| 649 + 5 files changed, 1440 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/273a08ed/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index a2832de..d4f2e53 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -430,6 +430,7 @@ minillaplocal.shared.query.files=alter_merge_2_orc.q,\ vectorized_dynamic_partition_pruning.q,\ vectorized_insert_into_bucketed_table.q,\ vectorized_mapjoin.q,\ + vectorized_mapjoin3.q,\ vectorized_math_funcs.q,\ vectorized_nested_mapjoin.q,\ vectorized_parquet.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/273a08ed/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java -- 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 8ce63f9..27b53b8 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 @@ -27,6 +27,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.IdentityHashMap; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -354,6 +356,8 @@ public class Vectorizer implements PhysicalPlanResolver { private BaseWork currentBaseWork; private Operator currentOperator; private Collection> vectorizedInputFormatExcludes; + private Map, Set, Operator>>> delayedFixups = + new IdentityHashMap, Set, Operator>>>(); public void testSetCurrentBaseWork(BaseWork testBaseWork) { currentBaseWork = testBaseWork; @@ -749,6 +753,8 @@ public class Vectorizer implements PhysicalPlanResolver { List> currentVectorParentList = newOperatorList(); currentVectorParentList.add(dummyVectorOperator); +delayedFixups.clear(); + do { List> nextParentList = newOperatorList(); List> nextVectorParentList= newOperatorList(); @@ -778,6 +784,8 @@ public class Vectorizer implements PhysicalPlanResolver { currentVectorParentList = nextVectorParentList; } while (currentParentList.size() > 0); +runDelayedFixups(); + return dummyVectorOperator; } @@ -844,12 +852,41 @@ public class Vectorizer implements PhysicalPlanResolver { if (childMultipleParent == parent) { childMultipleParents.set(i, vectorParent); } else { -fixupOtherParent(childMultipleParent, child, vectorChild); +queueDelayedFixup(childMultipleParent, child, vectorChild); } } vectorChild.setParentOperators(childMultipleParents); } + /* + * The fix up is delayed so that the parent operators aren't modified until the entire operator + * tree has been vectorized. + */ + private void queueDelayedFixup(Operator parent, + Operator child, Operator vectorChild) { +if (delayedFixups.get(parent) == null) { + HashSet, Operator>> value = + new HashSet, Operator>>(1); + delayedFixups.put(parent, value); +} +delayedFixups.get(parent).add( +new ImmutablePair, Operator>( +child, vectorChild)); + } + + private void runDelayedFixups() { +for (Entry, Set, Operator>>> delayed +: delayedFixups.entrySet()) { + Operator key = delayed.getKey(); + Set, Operator>> value = + delayed.getValue(); + for (ImmutablePair, Operator> swap : value) { +
hive git commit: HIVE-18688: Vectorization: Vectorizer Reason shouldn't be part of work-plan (Gopal V, reviewed by Ashutosh Chauhan)
Repository: hive Updated Branches: refs/heads/master 5ddd5851f -> 8cf36e733 HIVE-18688: Vectorization: Vectorizer Reason shouldn't be part of work-plan (Gopal V, reviewed by Ashutosh Chauhan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8cf36e73 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8cf36e73 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8cf36e73 Branch: refs/heads/master Commit: 8cf36e733471c760df173efaff3129dc46f7d0de Parents: 5ddd585 Author: Gopal V Authored: Tue Feb 13 10:25:22 2018 -0800 Committer: Gopal V Committed: Tue Feb 13 10:25:22 2018 -0800 -- ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/8cf36e73/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java index ae7cd57..dc3219b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java @@ -90,7 +90,7 @@ public abstract class BaseWork extends AbstractOperatorDesc { protected Set supportSetInUse; protected List supportRemovedReasons; - private VectorizerReason notVectorizedReason; + private transient VectorizerReason notVectorizedReason; private boolean groupByVectorOutput; private boolean allNative;
hive git commit: HIVE-18665: LLAP: Ignore cache-affinity if the LLAP IO elevator is disabled (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master 9a02aa86b -> 5ddd5851f HIVE-18665: LLAP: Ignore cache-affinity if the LLAP IO elevator is disabled (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5ddd5851 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5ddd5851 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5ddd5851 Branch: refs/heads/master Commit: 5ddd5851f179f265a7bf912656e1cc4c87a1a7a0 Parents: 9a02aa8 Author: Gopal V Authored: Tue Feb 13 10:23:01 2018 -0800 Committer: Gopal V Committed: Tue Feb 13 10:23:08 2018 -0800 -- .../hive/ql/exec/tez/HiveSplitGenerator.java| 10 ++-- .../apache/hadoop/hive/ql/exec/tez/Utils.java | 12 - .../org/apache/hadoop/hive/ql/plan/MapWork.java | 49 ++-- 3 files changed, 52 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/5ddd5851/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java -- 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 f3aa151..57f6c66 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 @@ -97,7 +97,8 @@ public class HiveSplitGenerator extends InputInitializer { // Assuming grouping enabled always. userPayloadProto = MRInputUserPayloadProto.newBuilder().setGroupingEnabled(true).build(); -this.splitLocationProvider = Utils.getSplitLocationProvider(conf, LOG); +this.splitLocationProvider = +Utils.getSplitLocationProvider(conf, work.getCacheAffinity(), LOG); LOG.info("SplitLocationProvider: " + splitLocationProvider); // Read all credentials into the credentials instance stored in JobConf. @@ -123,14 +124,15 @@ public class HiveSplitGenerator extends InputInitializer { this.jobConf = new JobConf(conf); -this.splitLocationProvider = Utils.getSplitLocationProvider(conf, LOG); -LOG.info("SplitLocationProvider: " + splitLocationProvider); - // Read all credentials into the credentials instance stored in JobConf. ShimLoader.getHadoopShims().getMergedCredentials(jobConf); this.work = Utilities.getMapWork(jobConf); +this.splitLocationProvider = +Utils.getSplitLocationProvider(conf, work.getCacheAffinity(), LOG); +LOG.info("SplitLocationProvider: " + splitLocationProvider); + // Events can start coming in the moment the InputInitializer is created. The pruner // must be setup and initialized here so that it sets up it's structures to start accepting events. // Setting it up in initialize leads to a window where events may come in before the pruner is http://git-wip-us.apache.org/repos/asf/hive/blob/5ddd5851/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java index b33f027..bc438bb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java @@ -32,11 +32,19 @@ import org.apache.hadoop.mapred.split.SplitLocationProvider; import org.slf4j.Logger; public class Utils { - public static SplitLocationProvider getSplitLocationProvider(Configuration conf, Logger LOG) throws + + public static SplitLocationProvider getSplitLocationProvider(Configuration conf, Logger LOG) + throws IOException { +// fall back to checking confs +return getSplitLocationProvider(conf, true, LOG); + } + + public static SplitLocationProvider getSplitLocationProvider(Configuration conf, boolean useCacheAffinity, Logger LOG) throws IOException { boolean useCustomLocations = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_MODE).equals("llap") -&& HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS); +&& HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS) +&& useCacheAffinity; SplitLocationProvider splitLocationProvider; LOG.info("SplitGenerator using llap affinitized locations: " + useCustomLocations); if (useCustomLocations) { http://git-wip-us.apache.org/repos/asf/hive/blob/5ddd5851/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java index
hive git commit: HIVE-18627: Errata
Repository: hive Updated Branches: refs/heads/master 0808f7d32 -> 6356205c7 HIVE-18627: Errata Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6356205c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6356205c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6356205c Branch: refs/heads/master Commit: 6356205c7cfd6cf8972ec3dce8cd89eae9433342 Parents: 0808f7d Author: Gopal V Authored: Mon Feb 12 19:33:53 2018 -0800 Committer: Gopal V Committed: Mon Feb 12 19:34:06 2018 -0800 -- errata.txt | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/6356205c/errata.txt -- diff --git a/errata.txt b/errata.txt index d1d95ef..cef9c50 100644 --- a/errata.txt +++ b/errata.txt @@ -1,6 +1,7 @@ Commits with the wrong or no JIRA referenced: git commit branch jira url +233884620af67e6af72b60629f799a69f5823eb2 master HIVE-18627 https://issues.apache.org/jira/browse/HIVE-18627 eb0034c0cdcc5f10fd5d7382e2caf787a8003e7a master HIVE-17420 https://issues.apache.org/jira/browse/HIVE-17420 f1aae85f197de09d4b86143f7f13d5aa21d2eb85 master HIVE-16431 https://issues.apache.org/jira/browse/HIVE-16431 cbab5b29f26ceb3d4633ade9647ce8bcb2f020a0 master HIVE-16422 https://issues.apache.org/jira/browse/HIVE-16422
hive git commit: PPD: Handle FLOAT boxing differently for single/double precision constants (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master 47cac2d0e -> 233884620 PPD: Handle FLOAT boxing differently for single/double precision constants (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/23388462 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/23388462 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/23388462 Branch: refs/heads/master Commit: 233884620af67e6af72b60629f799a69f5823eb2 Parents: 47cac2d Author: Gopal V Authored: Sun Feb 11 23:02:46 2018 -0800 Committer: Gopal V Committed: Sun Feb 11 23:02:46 2018 -0800 -- .../hive/ql/io/sarg/ConvertAstToSearchArg.java | 107 - .../test/queries/clientpositive/orc_ppd_basic.q | 17 +++ .../clientpositive/llap/orc_ppd_basic.q.out | 153 +++ 3 files changed, 240 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/23388462/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java -- 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 51b1ac6..27fe828 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 @@ -23,8 +23,6 @@ import java.sql.Timestamp; import java.util.List; import java.util.concurrent.ExecutionException; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.type.HiveChar; @@ -60,12 +58,35 @@ import org.slf4j.LoggerFactory; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; public class ConvertAstToSearchArg { private static final Logger LOG = LoggerFactory.getLogger(ConvertAstToSearchArg.class); private final SearchArgument.Builder builder; private final Configuration conf; + /* + * Create a new type for handling precision conversions from Decimal -> Double/Float + * + * The type is only relevant to boxLiteral and all other functions treat it identically. + */ + private static enum BoxType { +LONG(PredicateLeaf.Type.LONG), // all of the integer types +FLOAT(PredicateLeaf.Type.FLOAT), // float +DOUBLE(PredicateLeaf.Type.FLOAT), // double +STRING(PredicateLeaf.Type.STRING), // string, char, varchar +DATE(PredicateLeaf.Type.DATE), +DECIMAL(PredicateLeaf.Type.DECIMAL), +TIMESTAMP(PredicateLeaf.Type.TIMESTAMP), +BOOLEAN(PredicateLeaf.Type.BOOLEAN); + +public final PredicateLeaf.Type type; +BoxType(PredicateLeaf.Type type) { + this.type = type; +} + } + /** * Builds the expression and leaf list from the original predicate. * @param expression the expression to translate. @@ -89,7 +110,7 @@ public class ConvertAstToSearchArg { * @param expr the expression to get the type of * @return int, string, or float or null if we don't know the type */ - private static PredicateLeaf.Type getType(ExprNodeDesc expr) { + private static BoxType getType(ExprNodeDesc expr) { TypeInfo type = expr.getTypeInfo(); if (type.getCategory() == ObjectInspector.Category.PRIMITIVE) { switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) { @@ -97,22 +118,23 @@ public class ConvertAstToSearchArg { case SHORT: case INT: case LONG: - return PredicateLeaf.Type.LONG; + return BoxType.LONG; case CHAR: case VARCHAR: case STRING: - return PredicateLeaf.Type.STRING; + return BoxType.STRING; case FLOAT: + return BoxType.FLOAT; case DOUBLE: - return PredicateLeaf.Type.FLOAT; + return BoxType.DOUBLE; case DATE: - return PredicateLeaf.Type.DATE; + return BoxType.DATE; case TIMESTAMP: - return PredicateLeaf.Type.TIMESTAMP; + return BoxType.TIMESTAMP; case DECIMAL: - return PredicateLeaf.Type.DECIMAL; + return BoxType.DECIMAL; case BOOLEAN: - return PredicateLeaf.Type.BOOLEAN; + return BoxType.BOOLEAN; default: } } @@ -140,12 +162,12 @@ public class ConvertAstToSearchArg { } private static Object boxLiteral(ExprNodeConstantDesc constantDesc, - PredicateLeaf.Type type) { + BoxType boxType) { Object
hive git commit: HIVE-18597: LLAP: Always package the log4j2 API jar for org.apache.log4j (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master b9e36a4aa -> 3e85169a0 HIVE-18597: LLAP: Always package the log4j2 API jar for org.apache.log4j (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3e85169a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3e85169a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3e85169a Branch: refs/heads/master Commit: 3e85169a0876c9197b9c9e30279d3980467c6e79 Parents: b9e36a4 Author: Gopal V Authored: Mon Feb 5 14:28:36 2018 -0800 Committer: Gopal V Committed: Mon Feb 5 14:28:36 2018 -0800 -- .../java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3e85169a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java -- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java index 23531a3..3eaaed7 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java @@ -398,7 +398,7 @@ public class LlapServiceDriver { org.apache.logging.log4j.core.Appender.class, // log4j-core org.apache.logging.slf4j.Log4jLogger.class, // log4j-slf4j // log4j-1.2-API needed for NDC - org.apache.log4j.NDC.class, + org.apache.log4j.config.Log4j1ConfigurationFactory.class, io.netty.util.NetUtil.class, // netty4 org.jboss.netty.util.NetUtil.class //netty3 };
hive git commit: HIVE-18599: Transactions: Fix CTAS on Micromanaged tables (Steve Yeom, reviewed by Gopal V)
Repository: hive Updated Branches: refs/heads/master f9efd84f8 -> 47f45705f HIVE-18599: Transactions: Fix CTAS on Micromanaged tables (Steve Yeom, reviewed by Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/47f45705 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/47f45705 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/47f45705 Branch: refs/heads/master Commit: 47f45705f997658b3ce5df4276915a6c0be56d8a Parents: f9efd84 Author: Steve Yeom Authored: Fri Feb 2 16:48:02 2018 -0800 Committer: Gopal V Committed: Fri Feb 2 16:50:23 2018 -0800 -- .../test/resources/testconfiguration.properties | 1 + .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 13 ++- ql/src/test/queries/clientpositive/mm_cttas.q | 20 + .../results/clientpositive/llap/mm_cttas.q.out | 89 4 files changed, 120 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/47f45705/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 70d0749..fed9394 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -71,6 +71,7 @@ minillap.shared.query.files=insert_into1.q,\ mapreduce1.q,\ mapreduce2.q,\ mm_all.q,\ + mm_cttas.q,\ orc_merge1.q,\ orc_merge10.q,\ orc_merge2.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/47f45705/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java -- 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 dbf9363..c2e2499 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 @@ -2053,8 +2053,15 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { } } try { -fname = ctx.getExtTmpPathRelTo( -FileUtils.makeQualified(location, conf)).toString(); +CreateTableDesc tblDesc = qb.getTableDesc(); +if (tblDesc != null +&& tblDesc.isTemporary() +&& AcidUtils.isInsertOnlyTable(tblDesc.getTblProps(), true)) { + fname = FileUtils.makeQualified(location, conf).toString(); +} else { + fname = ctx.getExtTmpPathRelTo( + FileUtils.makeQualified(location, conf)).toString(); +} } catch (Exception e) { throw new SemanticException(generateErrorMessage(ast, "Error creating temporary folder on: " + location.toString()), e); @@ -6842,7 +6849,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { field_schemas = new ArrayList(); destTableIsTemporary = tblDesc.isTemporary(); destTableIsMaterialization = tblDesc.isMaterialization(); -if (!destTableIsTemporary && AcidUtils.isInsertOnlyTable(tblDesc.getTblProps(), true)) { +if (AcidUtils.isInsertOnlyTable(tblDesc.getTblProps(), true)) { isMmTable = isMmCtas = true; txnId = SessionState.get().getTxnMgr().getCurrentTxnId(); tblDesc.setInitialMmWriteId(txnId); http://git-wip-us.apache.org/repos/asf/hive/blob/47f45705/ql/src/test/queries/clientpositive/mm_cttas.q -- diff --git a/ql/src/test/queries/clientpositive/mm_cttas.q b/ql/src/test/queries/clientpositive/mm_cttas.q new file mode 100644 index 000..b099d2f --- /dev/null +++ b/ql/src/test/queries/clientpositive/mm_cttas.q @@ -0,0 +1,20 @@ + +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop table intermediate; +create table intermediate(key int) partitioned by (p int) stored as orc; +insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 2; +insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 2; +insert into table intermediate partition(p='457') select distinct key from src where key >= 100 order by key asc limit 2; + +drop table cttas1_mm; + +create temporary table cttas1_mm tblproperties ("transactional"="true", "transactional_properties"="insert_only") as select * from intermediate; + +select * from cttas1_mm; +
[2/2] hive git commit: HIVE-18529: Vectorization: Add a debug config option to disable scratch column reuse (Gopal V, reviewed by Matt McCline)
HIVE-18529: Vectorization: Add a debug config option to disable scratch column reuse (Gopal V, reviewed by Matt McCline) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d2a14fe2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d2a14fe2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d2a14fe2 Branch: refs/heads/master Commit: d2a14fe286234ba6a66716ad4e62bd11cf624e81 Parents: 8a6f36e Author: Gopal V Authored: Mon Jan 29 14:37:08 2018 -0800 Committer: Gopal V Committed: Mon Jan 29 14:40:17 2018 -0800 -- .../org/apache/hadoop/hive/conf/HiveConf.java |4 +- .../test/resources/testconfiguration.properties |3 +- .../resources/testconfiguration.properties.orig | 1640 -- .../ql/exec/vector/VectorizationContext.java| 22 +- .../clientpositive/vector_reuse_scratchcols.q | 79 + .../llap/vector_reuse_scratchcols.q.out | 412 + 6 files changed, 516 insertions(+), 1644 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d2a14fe2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.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 13067df..b7d3e99 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2952,7 +2952,9 @@ public class HiveConf extends Configuration { "internal use only, used for creating small group key vectorized row batches to exercise more logic\n" + "The default value is -1 which means don't restrict for testing", true), - + HIVE_VECTORIZATION_TESTING_REUSE_SCRATCH_COLUMNS("hive.vectorized.reuse.scratch.columns", true, + "internal use only. Disable this to debug scratch column state issues", + true), HIVE_VECTORIZATION_COMPLEX_TYPES_ENABLED("hive.vectorized.complex.types.enabled", true, "This flag should be set to true to enable vectorization\n" + "of expressions with complex types.\n" + http://git-wip-us.apache.org/repos/asf/hive/blob/d2a14fe2/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index c4676a7..d86ff58 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -733,7 +733,8 @@ minillaplocal.query.files=\ vector_partitioned_date_time.q,\ vector_ptf_1.q,\ vector_ptf_part_simple.q,\ - vector_udf_adaptor_1,\ + vector_reuse_scratchcols.q,\ + vector_udf_adaptor_1.q,\ vector_udf1.q,\ vector_windowing.q,\ vector_windowing_expressions.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/d2a14fe2/itests/src/test/resources/testconfiguration.properties.orig -- diff --git a/itests/src/test/resources/testconfiguration.properties.orig b/itests/src/test/resources/testconfiguration.properties.orig deleted file mode 100644 index f7ed4eb..000 --- a/itests/src/test/resources/testconfiguration.properties.orig +++ /dev/null @@ -1,1640 +0,0 @@ -# Note: the *.shared groups also run on TestCliDriver - -# NOTE: files should be listed in alphabetical order -minimr.query.files=infer_bucket_sort_map_operators.q,\ - infer_bucket_sort_dyn_part.q,\ - infer_bucket_sort_merge.q,\ - infer_bucket_sort_reducers_power_two.q,\ - infer_bucket_sort_num_buckets.q,\ - root_dir_external_table.q,\ - parallel_orderby.q,\ - bucket_num_reducers.q,\ - udf_using.q,\ - index_bitmap3.q,\ - index_bitmap_auto.q,\ - scriptfile1.q,\ - bucket_num_reducers2.q,\ - bucket_num_reducers_acid.q,\ - bucket_num_reducers_acid2.q,\ - scriptfile1_win.q - -# These tests are disabled for minimr -# ql_rewrite_gbtoidx.q,\ -# ql_rewrite_gbtoidx_cbo_1.q,\ -# ql_rewrite_gbtoidx_cbo_2.q,\ -# smb_mapjoin_8.q,\ - - -# Tests that are not enabled for CLI Driver -disabled.query.files=ql_rewrite_gbtoidx.q,\ - ql_rewrite_gbtoidx_cbo_1.q,\ - cbo_rp_subq_in.q,\ - cbo_rp_subq_not_in.q,\ - cbo_rp_subq_exists.q,\ - orc_llap.q,\ - ql_rewrite_gbtoidx_cbo_2.q,\ - rcfile_merge1.q,\ - stats_filemetadata.q,\ - mm_exim.q,\ - cbo_rp_insert.q,\ - cbo_rp_lineage2.q - -# NOTE: Add tests to minitez only if it is very -# specific to tez and cannot be added to minillap. -minitez.query.files.shared=delete_orig_table.q,\ - orc_merge12.q,\ - orc_vectorization_ppd.q,\ - update_orig_table.q,\ - vector_join_part_col_char.q,\ - vector_non_string_partition.q,\ - vectorization_div0.q,\ - vectori
[1/2] hive git commit: HIVE-18529: Vectorization: Add a debug config option to disable scratch column reuse (Gopal V, reviewed by Matt McCline)
Repository: hive Updated Branches: refs/heads/master 8a6f36e6f -> d2a14fe28 http://git-wip-us.apache.org/repos/asf/hive/blob/d2a14fe2/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out -- diff --git a/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out b/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out new file mode 100644 index 000..6528b6f --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/vector_reuse_scratchcols.q.out @@ -0,0 +1,412 @@ +PREHOOK: query: EXPLAIN VECTORIZATION DETAIL +SELECT AVG(cint), + (AVG(cint) + -3728), + (-((AVG(cint) + -3728))), + (-((-((AVG(cint) + -3728), + ((-((-((AVG(cint) + -3728) * (AVG(cint) + -3728)), + SUM(cdouble), + (-(AVG(cint))), + STDDEV_POP(cint), + (((-((-((AVG(cint) + -3728) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)), + STDDEV_SAMP(csmallint), + (-(STDDEV_POP(cint))), + (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)), + ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)) * STDDEV_POP(cint)), + VAR_SAMP(cint), + AVG(cfloat), + (10.175 - VAR_SAMP(cint)), + (-((10.175 - VAR_SAMP(cint, + ((-(STDDEV_POP(cint))) / -563), + STDDEV_SAMP(cint), + (-(((-(STDDEV_POP(cint))) / -563))), + (AVG(cint) / SUM(cdouble)), + MIN(ctinyint), + COUNT(csmallint), + (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)), + (-((AVG(cint) / SUM(cdouble +FROM alltypesorc +WHERE ((762 = cbigint) +OR ((csmallint < cfloat) +AND ((ctimestamp2 > -5) + AND (cdouble != cint))) +OR (cstring1 = 'a') + OR ((cbigint <= -1.389) + AND ((cstring2 != 'a') +AND ((79.553 != cint) + AND (cboolean2 != cboolean1) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL +SELECT AVG(cint), + (AVG(cint) + -3728), + (-((AVG(cint) + -3728))), + (-((-((AVG(cint) + -3728), + ((-((-((AVG(cint) + -3728) * (AVG(cint) + -3728)), + SUM(cdouble), + (-(AVG(cint))), + STDDEV_POP(cint), + (((-((-((AVG(cint) + -3728) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)), + STDDEV_SAMP(csmallint), + (-(STDDEV_POP(cint))), + (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)), + ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)) * STDDEV_POP(cint)), + VAR_SAMP(cint), + AVG(cfloat), + (10.175 - VAR_SAMP(cint)), + (-((10.175 - VAR_SAMP(cint, + ((-(STDDEV_POP(cint))) / -563), + STDDEV_SAMP(cint), + (-(((-(STDDEV_POP(cint))) / -563))), + (AVG(cint) / SUM(cdouble)), + MIN(ctinyint), + COUNT(csmallint), + (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)), + (-((AVG(cint) / SUM(cdouble +FROM alltypesorc +WHERE ((762 = cbigint) +OR ((csmallint < cfloat) +AND ((ctimestamp2 > -5) + AND (cdouble != cint))) +OR (cstring1 = 'a') + OR ((cbigint <= -1.389) + AND ((cstring2 != 'a') +AND ((79.553 != cint) + AND (cboolean2 != cboolean1) +POSTHOOK: type: QUERY +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 (CUSTOM_SIMPLE_EDGE) + A masked pattern was here + Vertices: +Map 1 +Map Operator Tree: +TableScan + alias: alltypesorc + Statistics: Num rows: 12288 Data size: 2601650 Basic stats: COMPLETE Column stats: COMPLETE + TableScan Vectorization: + native: true + vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct] + Filter Operator +Filter Vectorization: +className: VectorFilterOperator +native: true +predicateExpression: FilterExprOrExpr(children: FilterLongScalarEqualLongColumn(val 762, col 3:bigint), FilterExprAndExpr(children: FilterDoubleColLessDoubleColumn(col 13:float, col 4:float)(children: CastLongToFloatViaLongToDouble(col 1:smallint) -> 13:float), FilterDoubleColGreaterDoubleScalar(col 13:double, val -5.0)(children: CastTimestampToDouble(col 9:timestamp) -> 13:doub
hive git commit: HIVE-16821: Vectorization: support Explain Analyze in vectorized mode (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master 6dc245241 -> cb866e894 HIVE-16821: Vectorization: support Explain Analyze in vectorized mode (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cb866e89 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cb866e89 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cb866e89 Branch: refs/heads/master Commit: cb866e894bc5cf536ab3ba7b0e1542e8dbda7932 Parents: 6dc2452 Author: Gopal V Authored: Wed Jan 24 22:35:26 2018 -0800 Committer: Gopal V Committed: Wed Jan 24 22:35:26 2018 -0800 -- .../apache/hadoop/hive/ql/exec/Operator.java| 46 +--- .../hadoop/hive/ql/exec/TableScanOperator.java | 35 +++- .../VectorReduceSinkCommonOperator.java | 1 + .../optimizer/physical/PhysicalOptimizer.java | 3 +- .../hadoop/hive/ql/parse/TezCompiler.java | 3 +- .../hive/ql/parse/spark/SparkCompiler.java | 3 +- .../queries/clientpositive/explainanalyze_3.q | 1 + .../clientpositive/tez/explainanalyze_3.q.out | 56 ++-- 8 files changed, 83 insertions(+), 65 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/cb866e89/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java index 2462938..199b181 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java @@ -113,9 +113,8 @@ public abstract class Operator implements Serializable,C private boolean useBucketizedHiveInputFormat; // Data structures specific for vectorized operators. - private int size; - private boolean selectedInUse; - private int[] selected; + private transient boolean multiChildren; + private transient int[] selected; // dummy operator (for not increasing seqId) protected Operator(String name, CompilationOpContext cContext) { @@ -129,8 +128,6 @@ public abstract class Operator implements Serializable,C childOperators = new ArrayList>(); parentOperators = new ArrayList>(); abortOp = new AtomicBoolean(false); -// Initializing data structures for vectorization -selected = new int[VectorizedRowBatch.DEFAULT_SIZE]; } public Operator(CompilationOpContext cContext) { @@ -323,6 +320,9 @@ public abstract class Operator implements Serializable,C // String className = this.getClass().getName(); this.done = false; +this.runTimeNumRows = 0; // initializeOp can be overridden +// Initializing data structures for vectorForward +this.selected = new int[VectorizedRowBatch.DEFAULT_SIZE]; if (state == State.INIT) { return; } @@ -345,6 +345,7 @@ public abstract class Operator implements Serializable,C for (int i = 0; i < childOperatorsArray.length; i++) { childOperatorsArray[i] = childOperators.get(i); } +multiChildren = childOperatorsArray.length > 1; childOperatorsTag = new int[childOperatorsArray.length]; for (int i = 0; i < childOperatorsArray.length; i++) { List> parentOperators = @@ -487,7 +488,6 @@ public abstract class Operator implements Serializable,C protected void initializeOp(Configuration hconf) throws HiveException { this.hconf = hconf; rootInitializeCalled = true; -runTimeNumRows = 0; } /** @@ -704,6 +704,12 @@ public abstract class Operator implements Serializable,C // call the operator specific close routine closeOp(abort); +// closeOp can be overriden +if (conf != null && conf.getRuntimeStatsTmpDir() != null) { + publishRunTimeStats(); +} +this.runTimeNumRows = 0; + reporter = null; try { @@ -733,10 +739,6 @@ public abstract class Operator implements Serializable,C * should overwrite this funtion for their specific cleanup routine. */ protected void closeOp(boolean abort) throws HiveException { -if (conf != null && conf.getRuntimeStatsTmpDir() != null) { - publishRunTimeStats(); -} -runTimeNumRows = 0; } private boolean jobCloseDone = false; @@ -894,26 +896,32 @@ public abstract class Operator implements Serializable,C forward(row, rowInspector, false); } + protected void forward(VectorizedRowBatch vrg, ObjectInspector rowInspector) + throws HiveException { +forward(vrg, rowInspector, true); + } + protected void forward(Object row, ObjectInspector rowInspector, boolean isVectorized) throws HiveException { -if (isVectorized && getNumChild() > 1) { +if (isVectorized) { vectorForward((VectorizedRowBatch) row, rowInsp
hive git commit: HIVE-18213: Tests: YARN Minicluster times out if the disks are >90% full (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master 2a89e83c2 -> d041cc441 HIVE-18213: Tests: YARN Minicluster times out if the disks are >90% full (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d041cc44 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d041cc44 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d041cc44 Branch: refs/heads/master Commit: d041cc441e614e2e9a12645d9775803c6359bfad Parents: 2a89e83 Author: Gopal V Authored: Mon Dec 4 22:57:09 2017 -0800 Committer: Gopal V Committed: Mon Dec 4 22:57:09 2017 -0800 -- data/conf/hive-site.xml | 6 ++ data/conf/llap/hive-site.xml | 5 + data/conf/spark/yarn-client/hive-site.xml | 5 + data/conf/tez/hive-site.xml | 5 + 4 files changed, 21 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d041cc44/data/conf/hive-site.xml -- diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml index a205b8c..6908bb1 100644 --- a/data/conf/hive-site.xml +++ b/data/conf/hive-site.xml @@ -319,4 +319,10 @@ true + + + yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage + 99 + + http://git-wip-us.apache.org/repos/asf/hive/blob/d041cc44/data/conf/llap/hive-site.xml -- diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml index 8cd5144..cdda875 100644 --- a/data/conf/llap/hive-site.xml +++ b/data/conf/llap/hive-site.xml @@ -343,4 +343,9 @@ true + + yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage + 99 + + http://git-wip-us.apache.org/repos/asf/hive/blob/d041cc44/data/conf/spark/yarn-client/hive-site.xml -- diff --git a/data/conf/spark/yarn-client/hive-site.xml b/data/conf/spark/yarn-client/hive-site.xml index bebca88..64b56a5 100644 --- a/data/conf/spark/yarn-client/hive-site.xml +++ b/data/conf/spark/yarn-client/hive-site.xml @@ -295,4 +295,9 @@ 3ms + + yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage + 99 + + http://git-wip-us.apache.org/repos/asf/hive/blob/d041cc44/data/conf/tez/hive-site.xml -- diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml index f1dabf5..4519678 100644 --- a/data/conf/tez/hive-site.xml +++ b/data/conf/tez/hive-site.xml @@ -288,4 +288,9 @@ true + + yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage + 99 + +
hive git commit: HIVE-17399: Semijoin: Do not remove semijoin branch if it feeds to TS->DPP_EVENT (Deepak Jaiswal, reviewed by Gopal V)
Repository: hive Updated Branches: refs/heads/master b82d38aa3 -> 2e226d22f HIVE-17399: Semijoin: Do not remove semijoin branch if it feeds to TS->DPP_EVENT (Deepak Jaiswal, reviewed by Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2e226d22 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2e226d22 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2e226d22 Branch: refs/heads/master Commit: 2e226d22facea6ea362dc90ce37ae59aeb1666a0 Parents: b82d38a Author: Deepak Jaiswal Authored: Fri Sep 1 10:51:52 2017 -0700 Committer: Gopal V Committed: Fri Sep 1 10:52:44 2017 -0700 -- .../hive/ql/parse/SemiJoinBranchInfo.java | 18 + .../hadoop/hive/ql/parse/TezCompiler.java | 105 +- .../clientpositive/dynamic_semijoin_reduction.q | 10 + .../llap/dynamic_semijoin_reduction.q.out | 365 +++ 4 files changed, 490 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/2e226d22/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java index 5d7b9e5..c960b05 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java @@ -24,15 +24,21 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator; public class SemiJoinBranchInfo { private TableScanOperator ts; private boolean isHint; + // Default value is true, however, if an optimization deems this edge + // important, it should set this to false. This does not guarantee that + // the edge will stay, however, it increases the chances. + private boolean shouldRemove; public SemiJoinBranchInfo(TableScanOperator ts) { this.ts = ts; isHint = false; +shouldRemove = true; } public SemiJoinBranchInfo(TableScanOperator ts, boolean isHint) { this.ts = ts; this.isHint = isHint; +shouldRemove = !isHint; // If hint is true, shouldRemove is redundant anyway } public TableScanOperator getTsOp() { @@ -42,4 +48,16 @@ public class SemiJoinBranchInfo { public boolean getIsHint() { return isHint; } + + public boolean getShouldRemove() { +return shouldRemove; + } + + public void setShouldRemove(boolean shouldRemove) { +// The state only changes from true->false +// Once set to false, it may not change back to true +if (this.shouldRemove) { + this.shouldRemove = shouldRemove; +} + } } http://git-wip-us.apache.org/repos/asf/hive/blob/2e226d22/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java -- 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 5921594..15836ec 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 @@ -165,8 +165,14 @@ public class TezCompiler extends TaskCompiler { runRemoveDynamicPruningOptimization(procCtx, inputs, outputs); perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Run remove dynamic pruning by size"); +perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER); +markSemiJoinForDPP(procCtx); +perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Mark certain semijoin edges important based "); + // Removing semijoin optimization when it may not be beneficial +perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER); removeSemijoinOptimizationByBenefit(procCtx); +perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Remove Semijoins based on cost benefits"); perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER); // Remove any parallel edge between semijoin and mapjoin. @@ -955,12 +961,14 @@ public class TezCompiler extends TaskCompiler { if (ts.getStatistics() != null) { long numRows = ts.getStatistics().getNumRows(); if (numRows < pCtx.getConf().getLongVar(ConfVars.TEZ_BIGTABLE_MIN_SIZE_SEMIJOIN_REDUCTION)) { - if (LOG.isDebugEnabled()) { -LOG.debug("Insufficient rows (" + numRows + ") to justify semijoin optimization. Removing semijoin " -+ OperatorUtils.getOpNamePretty(rs) + " - " + OperatorUtils.getOpNamePretty(ts)); + if (sjInfo.getShouldRemove()) { +if (LOG.isDebugEnabled()) { + LOG
hive git commit: HIVE-17194: JDBC: Implement Gzip compression for HTTP mode (Gopal V, reviewed by Thejas Nair)
Repository: hive Updated Branches: refs/heads/master 48be6aecd -> c9e09400f HIVE-17194: JDBC: Implement Gzip compression for HTTP mode (Gopal V, reviewed by Thejas Nair) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c9e09400 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c9e09400 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c9e09400 Branch: refs/heads/master Commit: c9e09400ff7f2ca662acfaedf3cbf327efb3d00f Parents: 48be6ae Author: Gopal V Authored: Thu Aug 17 22:25:06 2017 -0700 Committer: Gopal V Committed: Thu Aug 17 22:25:06 2017 -0700 -- .../org/apache/hadoop/hive/conf/HiveConf.java| 4 +++- .../service/cli/thrift/ThriftHttpCLIService.java | 19 --- 2 files changed, 19 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/c9e09400/common/src/java/org/apache/hadoop/hive/conf/HiveConf.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 3c158a6..33149d6 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2430,7 +2430,9 @@ public class HiveConf extends Configuration { HIVE_SERVER2_THRIFT_HTTP_REQUEST_HEADER_SIZE("hive.server2.thrift.http.request.header.size", 6*1024, "Request header size in bytes, when using HTTP transport mode. Jetty defaults used."), HIVE_SERVER2_THRIFT_HTTP_RESPONSE_HEADER_SIZE("hive.server2.thrift.http.response.header.size", 6*1024, -"Response header size in bytes, when using HTTP transport mode. Jetty defaults used."), +"Response header size in bytes, when using HTTP transport mode. Jetty defaults used."), + HIVE_SERVER2_THRIFT_HTTP_COMPRESSION_ENABLED("hive.server2.thrift.http.compression.enabled", true, +"Enable thrift http compression via Jetty compression support"), // Cookie based authentication when using HTTP Transport HIVE_SERVER2_THRIFT_HTTP_COOKIE_AUTH_ENABLED("hive.server2.thrift.http.cookie.auth.enabled", true, http://git-wip-us.apache.org/repos/asf/hive/blob/c9e09400/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java -- 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 c4d4e02..828486b 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 @@ -23,6 +23,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.TimeUnit; +import javax.ws.rs.HttpMethod; + import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; @@ -42,7 +44,7 @@ import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; -import org.eclipse.jetty.servlet.FilterMapping; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ssl.SslContextFactory; @@ -50,6 +52,8 @@ import org.eclipse.jetty.util.thread.ExecutorThreadPool; public class ThriftHttpCLIService extends ThriftCLIService { + private static final String APPLICATION_THRIFT = "application/x-thrift"; + private final Runnable oomHook; public ThriftHttpCLIService(CLIService cliService, Runnable oomHook) { @@ -149,9 +153,18 @@ public class ThriftHttpCLIService extends ThriftCLIService { LOG.warn("XSRF filter disabled"); } - String httpPath = getHttpPath(hiveConf + final String httpPath = getHttpPath(hiveConf .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH)); - httpServer.setHandler(context); + + if (HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_SERVER2_THRIFT_HTTP_COMPRESSION_ENABLED)) { +final GzipHandler gzipHandler = new GzipHandler(); +gzipHandler.setHandler(context); +gzipHandler.addIncludedMethods(HttpMethod.POST); +gzipHandler.addIncludedMimeTypes(APPLICATION_THRIFT); +httpServer.setHandler(gzipHandler); + } else { +httpServer.setHandler(context); + } context.addServlet(new ServletHolder(thriftHttpServlet), httpPath); // TODO: check defaults
hive git commit: HIVE-16801: Vectorization: throwExpandError should be an immediate fatal (Gopal V, reviewed by Prasanth Jayachandran)
Repository: hive Updated Branches: refs/heads/master b560f492e -> 3f5e65288 HIVE-16801: Vectorization: throwExpandError should be an immediate fatal (Gopal V, reviewed by Prasanth Jayachandran) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3f5e6528 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3f5e6528 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3f5e6528 Branch: refs/heads/master Commit: 3f5e65288eacbff9d2c373d3bd4e1eaa74517f67 Parents: b560f49 Author: Gopal V Authored: Thu Jun 1 22:59:50 2017 -0700 Committer: Gopal V Committed: Thu Jun 1 22:59:50 2017 -0700 -- .../ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/3f5e6528/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java index 1f182ee..ce9a962 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast; import org.apache.hadoop.hive.ql.util.JavaDataModel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashTable; public abstract class VectorMapJoinFastHashTable implements VectorMapJoinHashTable { @@ -47,7 +48,7 @@ public abstract class VectorMapJoinFastHashTable implements VectorMapJoinHashTab public static final int ONE_SIXTH_LIMIT = HIGHEST_INT_POWER_OF_2 / 6; public void throwExpandError(int limit, String dataTypeName) { -throw new RuntimeException( +throw new MapJoinMemoryExhaustionError( "Vector MapJoin " + dataTypeName + " Hash Table cannot grow any more -- use a smaller container size. " + "Current logical size is " + logicalHashBucketCount + " and " + "the limit is " + limit + ". " +
[2/2] hive git commit: HIVE-16592: Vectorization: Long hashCodes should bit-mix into lower bits (Gopal V, reviewed by Sergey Shelukhin)
HIVE-16592: Vectorization: Long hashCodes should bit-mix into lower bits (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/538c0088 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/538c0088 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/538c0088 Branch: refs/heads/master Commit: 538c0088ab89cdcc30f4d1ae2c7d2bd266d18235 Parents: 40fe0d7 Author: Gopal V Authored: Fri May 12 00:11:35 2017 -0700 Committer: Gopal V Committed: Fri May 12 00:11:44 2017 -0700 -- .../apache/hive/common/util/HashCodeUtil.java | 30 ++-- .../hive/common/type/FastHiveDecimalImpl.java | 2 +- 2 files changed, 9 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/538c0088/common/src/java/org/apache/hive/common/util/HashCodeUtil.java -- diff --git a/common/src/java/org/apache/hive/common/util/HashCodeUtil.java b/common/src/java/org/apache/hive/common/util/HashCodeUtil.java index fa30273..b47479b 100644 --- a/common/src/java/org/apache/hive/common/util/HashCodeUtil.java +++ b/common/src/java/org/apache/hive/common/util/HashCodeUtil.java @@ -34,33 +34,19 @@ public class HashCodeUtil { } public static int calculateLongHashCode(long key) { - -key = (~key) + (key << 21); // key = (key << 21) - key - 1; -key = key ^ (key >>> 24); -key = (key + (key << 3)) + (key << 8); // key * 265 -key = key ^ (key >>> 14); -key = (key + (key << 2)) + (key << 4); // key * 21 -key = key ^ (key >>> 28); -key = key + (key << 31); - +// Mixing down into the lower bits - this produces a worse hashcode in purely +// numeric terms, but leaving entropy in the higher bits is not useful for a +// 2^n bucketing scheme. See JSR166 ConcurrentHashMap r1.89 (released under Public Domain) +// Note: ConcurrentHashMap has since reverted this to retain entropy bits higher +// up, to support the 2-level hashing for segment which operates at a higher bitmask +key ^= (key >>> 7) ^ (key >>> 4); +key ^= (key >>> 20) ^ (key >>> 12); return (int) key; } public static void calculateLongArrayHashCodes(long[] longs, int[] hashCodes, final int count) { -long key; for (int v = 0; v < count; v++) { - - key = longs[v]; - - // Hash code logic from calculateLongHashCode. - key = (~key) + (key << 21); // key = (key << 21) - key - 1; - key = key ^ (key >>> 24); - key = (key + (key << 3)) + (key << 8); // key * 265 - key = key ^ (key >>> 14); - key = (key + (key << 2)) + (key << 4); // key * 21 - key = key ^ (key >>> 28); - key = key + (key << 31); - hashCodes[v] = (int) key; + hashCodes[v] = (int) calculateLongHashCode(longs[v]); } } http://git-wip-us.apache.org/repos/asf/hive/blob/538c0088/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java -- 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 88abf3c..ef9cbcf 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 @@ -3913,7 +3913,7 @@ public class FastHiveDecimalImpl extends FastHiveDecimal { long key = fast0; -// Hash code logic from calculateLongHashCode. +// Hash code logic from original calculateLongHashCode key = (~key) + (key << 21); // key = (key << 21) - key - 1; key = key ^ (key >>> 24);
[1/2] hive git commit: HIVE-16635: Progressbar: Use different timeouts for running queries (Gopal V, reviewed by Siddharth Seth)
Repository: hive Updated Branches: refs/heads/master 6bfa2491b -> 538c0088a HIVE-16635: Progressbar: Use different timeouts for running queries (Gopal V, reviewed by Siddharth Seth) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/40fe0d7e Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/40fe0d7e Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/40fe0d7e Branch: refs/heads/master Commit: 40fe0d7e03a03bf6082ff00f17322756e6f00ea9 Parents: 6bfa249 Author: Gopal V Authored: Fri May 12 00:07:22 2017 -0700 Committer: Gopal V Committed: Fri May 12 00:07:22 2017 -0700 -- .../ql/exec/tez/monitoring/TezJobMonitor.java | 30 +--- 1 file changed, 26 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/40fe0d7e/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java -- 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 f2f97f3..049d7fd 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec.tez.monitoring; import com.google.common.base.Preconditions; + import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; @@ -39,6 +40,7 @@ import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGStatus; import org.apache.tez.dag.api.client.Progress; import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.util.StopWatch; import java.io.IOException; import java.io.InterruptedIOException; @@ -47,6 +49,7 @@ import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING; @@ -58,8 +61,10 @@ import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING; public class TezJobMonitor { static final String CLASS_NAME = TezJobMonitor.class.getName(); - private static final int CHECK_INTERVAL = 200; + private static final int MIN_CHECK_INTERVAL = 200; + private static final int MAX_CHECK_INTERVAL = 1000; 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 static final List shutdownList; @@ -124,6 +129,7 @@ public class TezJobMonitor { boolean done = false; boolean success = false; int failedCounter = 0; +final StopWatch failureTimer = new StopWatch(); int rc = 0; DAGStatus status = null; Map vertexProgressMap = null; @@ -138,6 +144,7 @@ public class TezJobMonitor { DAGStatus.State lastState = null; boolean running = false; +int checkInterval = MIN_CHECK_INTERVAL; while (true) { try { @@ -145,10 +152,13 @@ public class TezJobMonitor { context.checkHeartbeaterLockException(); } -status = dagClient.getDAGStatus(new HashSet(), CHECK_INTERVAL); +status = dagClient.getDAGStatus(new HashSet(), checkInterval); vertexProgressMap = status.getVertexProgress(); DAGStatus.State state = status.getState(); +failedCounter = 0; // AM is responsive again (recovery?) +failureTimer.reset(); + if (state != lastState || state == RUNNING) { lastState = state; @@ -166,6 +176,8 @@ public class TezJobMonitor { console.printInfo("Status: Running (" + dagClient.getExecutionContext() + ")\n"); this.executionStartTime = System.currentTimeMillis(); running = true; +// from running -> failed/succeeded, the AM breaks out of timeouts +checkInterval = MAX_CHECK_INTERVAL; } updateFunction.update(status, vertexProgressMap); break; @@ -204,9 +216,19 @@ public class TezJobMonitor { } catch (Exception e) { console.printInfo("Exception: " + e.getMessage()); boolean isInterrupted = hasInterruptedException(e); -if (isInterrupted || (++failedCounter % MAX_RETRY_INTERVAL / CHECK_INTERVAL == 0)) { +if (failedCounter == 0) { + failureTimer.reset(); + failureTimer.start(); +} +if (isInterrupted +|| (++failedCounter >= MAX_RETRY_FAILURES && failureTimer.now(TimeUnit.MILLIS
hive git commit: HIVE-16533: Vectorization: Avoid evaluating empty groupby keys (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master 1cae318fb -> e35b3f253 HIVE-16533: Vectorization: Avoid evaluating empty groupby keys (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e35b3f25 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e35b3f25 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e35b3f25 Branch: refs/heads/master Commit: e35b3f253d87fdc5120e733a8654b2aa85884e4e Parents: 1cae318 Author: Gopal V Authored: Wed Apr 26 20:10:30 2017 -0700 Committer: Gopal V Committed: Wed Apr 26 20:10:30 2017 -0700 -- .../hadoop/hive/ql/exec/vector/VectorGroupByOperator.java | 10 +- .../hive/ql/exec/vector/VectorHashKeyWrapperBatch.java| 6 ++ 2 files changed, 15 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e35b3f25/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java -- 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 4b76d74..5b4c7c3 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 @@ -458,10 +458,18 @@ public class VectorGroupByOperator extends Operator implements // to bump its internal version. aggregationBatchInfo.startBatch(); + if (batch.size == 0) { +return; + } + // We now have to probe the global hash and find-or-allocate // the aggregation buffers to use for each key present in the batch VectorHashKeyWrapper[] keyWrappers = keyWrappersBatch.getVectorHashKeyWrappers(); - for (int i=0; i < batch.size; ++i) { + + final int n = keyExpressions.length == 0 ? 1 : batch.size; + // note - the row mapping is not relevant when aggregationBatchInfo::getDistinctBufferSetCount() == 1 + + for (int i=0; i < n; ++i) { VectorHashKeyWrapper kw = keyWrappers[i]; VectorAggregationBufferRow aggregationBuffer = mapKeysAggregationBuffers.get(kw); if (null == aggregationBuffer) { http://git-wip-us.apache.org/repos/asf/hive/blob/e35b3f25/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java index 63cdf94..b235a3e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec.vector; +import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper.EmptyVectorHashKeyWrapper; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter; import org.apache.hadoop.hive.ql.metadata.HiveException; @@ -89,6 +90,11 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo { */ public void evaluateBatch(VectorizedRowBatch batch) throws HiveException { +if (keyCount == 0) { + // all keywrappers must be EmptyVectorHashKeyWrapper + return; +} + for(int i=0;i
hive git commit: HIVE-16353: Jetty 9 upgrade breaks hive master LLAP (Gopal V, reviewed by Rajesh Balamohan)
Repository: hive Updated Branches: refs/heads/master 120271e92 -> 1cae318fb HIVE-16353: Jetty 9 upgrade breaks hive master LLAP (Gopal V, reviewed by Rajesh Balamohan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1cae318f Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1cae318f Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1cae318f Branch: refs/heads/master Commit: 1cae318fb09a4b3210c8fb3c233811bd000c829d Parents: 120271e Author: Gopal V Authored: Wed Apr 26 20:00:52 2017 -0700 Committer: Gopal V Committed: Wed Apr 26 20:00:52 2017 -0700 -- .../java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java| 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1cae318f/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java -- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java index 4a4fef0..a662c75 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java @@ -76,6 +76,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.eclipse.jetty.rewrite.handler.Rule; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.joda.time.DateTime; import org.json.JSONException; @@ -379,6 +380,7 @@ public class LlapServiceDriver { LlapInputFormat.class, // llap-server HiveInputFormat.class, // hive-exec SslContextFactory.class, // hive-common (https deps) + Rule.class, // Jetty rewrite class RegistryUtils.ServiceRecordMarshal.class, // ZK registry // log4j2 com.lmax.disruptor.RingBuffer.class, // disruptor
hive git commit: HIVE-16329: TopN: use local executor info for LLAP memory checks (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master c383ee30c -> cc1f4e4a2 HIVE-16329: TopN: use local executor info for LLAP memory checks (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cc1f4e4a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cc1f4e4a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cc1f4e4a Branch: refs/heads/master Commit: cc1f4e4a2451edf8ec5d9abd0cabfdcad8d55d16 Parents: c383ee3 Author: Gopal V Authored: Tue Apr 25 11:41:43 2017 -0700 Committer: Gopal V Committed: Tue Apr 25 11:41:43 2017 -0700 -- .../apache/hadoop/hive/llap/LlapDaemonInfo.java | 92 .../hive/llap/daemon/impl/LlapDaemon.java | 23 ++--- .../hive/llap/daemon/MiniLlapCluster.java | 5 ++ .../hadoop/hive/ql/exec/GroupByOperator.java| 5 +- .../apache/hadoop/hive/ql/exec/TopNHash.java| 5 +- 5 files changed, 116 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/cc1f4e4a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapDaemonInfo.java -- diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/LlapDaemonInfo.java b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapDaemonInfo.java new file mode 100644 index 000..fa29b59 --- /dev/null +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/LlapDaemonInfo.java @@ -0,0 +1,92 @@ +/* + * Licensed 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.llap; + +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.yarn.api.records.Resource; + +public enum LlapDaemonInfo { + INSTANCE; + + private static final class LlapDaemonInfoHolder { +public LlapDaemonInfoHolder(int numExecutors, long executorMemory, long cacheSize, +boolean isDirectCache, boolean isLlapIo) { + this.numExecutors = numExecutors; + this.executorMemory = executorMemory; + this.cacheSize = cacheSize; + this.isDirectCache = isDirectCache; + this.isLlapIo = isLlapIo; +} + +final int numExecutors; +final long executorMemory; +final long cacheSize; +final boolean isDirectCache; +final boolean isLlapIo; + } + + // add more variables as required + private AtomicReference dataRef = + new AtomicReference(); + + public static void initialize(String appName, Configuration daemonConf) { +int numExecutors = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_NUM_EXECUTORS); +long executorMemoryBytes = +HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB) * 1024l * 1024l; +long ioMemoryBytes = HiveConf.getSizeVar(daemonConf, ConfVars.LLAP_IO_MEMORY_MAX_SIZE); +boolean isDirectCache = HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_ALLOCATOR_DIRECT); +boolean isLlapIo = HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.LLAP_IO_ENABLED, true); +initialize(appName, numExecutors, executorMemoryBytes, ioMemoryBytes, isDirectCache, isLlapIo); + } + + public static void initialize(String appName, int numExecutors, long executorMemoryBytes, + long ioMemoryBytes, boolean isDirectCache, boolean isLlapIo) { +INSTANCE.dataRef.set(new LlapDaemonInfoHolder(numExecutors, executorMemoryBytes, ioMemoryBytes, +isDirectCache, isLlapIo)); + } + + public boolean isLlap() { +return dataRef.get() != null; + } + + public int getNumExecutors() { +return dataRef.get().numExecutors; + } + + public long getExecutorMemory() { +return dataRef.get().executorMemory; + } + + public long getMemoryPerExecutor() { +final LlapDaemonInfoHolder data = dataRef.get(); +return (getExecutorMemory() - -(data.isDirectCache ? 0 : data.cacheSize)) / getNumExecutors(); + } + + public long getCacheSize() { +return dataRef.get().cacheSize; + } + + public boolean isDirectCache() { +return dataRef.get().isDirectCache; + } + + public boolean isLlapIo() { +return dataRef.get().isLlapIo; + } + +} http://git-wip-us.apache.org/repos/asf/hive/blob/cc1f4e4a
hive git commit: HIVE-16425: Vectorization: unload old hashtables before reloadHashTable (Gopal V, reviewed by Matt McCline) (addendum)
Repository: hive Updated Branches: refs/heads/master e6143de2b -> f66b09a61 HIVE-16425: Vectorization: unload old hashtables before reloadHashTable (Gopal V, reviewed by Matt McCline) (addendum) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f66b09a6 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f66b09a6 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f66b09a6 Branch: refs/heads/master Commit: f66b09a61bbeaaffaa2572c2806d5fb05292e753 Parents: e6143de Author: Gopal V Authored: Wed Apr 12 23:14:15 2017 +0530 Committer: Gopal V Committed: Wed Apr 12 23:14:15 2017 +0530 -- errata.txt | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/f66b09a6/errata.txt -- diff --git a/errata.txt b/errata.txt index 6f464cf..e97f416 100644 --- a/errata.txt +++ b/errata.txt @@ -1,6 +1,7 @@ Commits with the wrong or no JIRA referenced: git commit branch jira url +e6143de2b0c3f53d32db8a743119e3a8080d4f85 master HIVE-16425 https://issues.apache.org/jira/browse/HIVE-16425 3f90794d872e90c29a068f16cdf3f45b1cf52c74 master HIVE-15579 https://issues.apache.org/jira/browse/HIVE-15579 5a576b6fbf1680ab4dd8f275cad484a2614ef2c1 master HIVE-10391 https://issues.apache.org/jira/browse/HIVE-10391 582f4e1bc39b9605d11f762480b29561a44688ae llap HIVE-10217 https://issues.apache.org/jira/browse/HIVE-10217
hive git commit: Vectorization: unload old hashtables before reloadHashTable (Gopal V, reviewed by Matt McCline)
Repository: hive Updated Branches: refs/heads/master bd313cf36 -> e6143de2b Vectorization: unload old hashtables before reloadHashTable (Gopal V, reviewed by Matt McCline) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e6143de2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e6143de2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e6143de2 Branch: refs/heads/master Commit: e6143de2b0c3f53d32db8a743119e3a8080d4f85 Parents: bd313cf Author: Gopal V Authored: Wed Apr 12 23:12:04 2017 +0530 Committer: Gopal V Committed: Wed Apr 12 23:12:04 2017 +0530 -- .../exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java | 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e6143de2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java index cb30413..c4d5113 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java @@ -531,6 +531,8 @@ public abstract class VectorMapJoinGenerateResultOperator extends VectorMapJoinC protected void reloadHashTable(byte pos, int partitionId) throws IOException, HiveException, SerDeException, ClassNotFoundException { +this.vectorMapJoinHashTable = null; + // The super method will reload a hash table partition of one of the small tables. // Currently, for native vector map join it will only be one small table. super.reloadHashTable(pos, partitionId);
hive git commit: HIVE-16208: Vectorization: ProcessingModeHashAggregate::sumBatchSize is never reset (Gopal V, reviewed by Rajesh Balamohan)
Repository: hive Updated Branches: refs/heads/master 24f1861e8 -> 60795505b HIVE-16208: Vectorization: ProcessingModeHashAggregate::sumBatchSize is never reset (Gopal V, reviewed by Rajesh Balamohan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/60795505 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/60795505 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/60795505 Branch: refs/heads/master Commit: 60795505b5c8ff1162afcb7e93013665b228e150 Parents: 24f1861 Author: Gopal V Authored: Thu Mar 23 23:05:05 2017 +0530 Committer: Gopal V Committed: Thu Mar 23 23:05:18 2017 +0530 -- .../apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java | 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/60795505/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java -- 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 90748e9..86d964c 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 @@ -374,6 +374,8 @@ public class VectorGroupByOperator extends Operator implements HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL.defaultIntVal; } + sumBatchSize = 0; + mapKeysAggregationBuffers = new HashMap(); computeMemoryLimits(); LOG.debug("using hash aggregation processing mode");
hive git commit: HIVE-15789: Vectorization: limit reduce vectorization to 32Mb chunks (Teddy Choi, via Gopal V)
Repository: hive Updated Branches: refs/heads/master ea3be9549 -> e434f8320 HIVE-15789: Vectorization: limit reduce vectorization to 32Mb chunks (Teddy Choi, via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e434f832 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e434f832 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e434f832 Branch: refs/heads/master Commit: e434f832015a38eecb7ac7cb5d45fbc8a2ea95ca Parents: ea3be95 Author: Teddy Choi Authored: Thu Mar 23 02:24:36 2017 +0530 Committer: Gopal V Committed: Thu Mar 23 02:25:29 2017 +0530 -- .../hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java | 6 +- .../apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java| 7 ++- .../apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java | 5 + 3 files changed, 16 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e434f832/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java index 27bed9c..7eaad18 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java @@ -95,6 +95,7 @@ public class SparkReduceRecordHandler extends SparkRecordHandler { // number of columns pertaining to keys in a vectorized row batch private int keysColumnOffset; private static final int BATCH_SIZE = VectorizedRowBatch.DEFAULT_SIZE; + private static final int BATCH_BYTES = VectorizedRowBatch.DEFAULT_BYTES; private StructObjectInspector keyStructInspector; private StructObjectInspector[] valueStructInspectors; /* this is only used in the error code path */ @@ -373,6 +374,7 @@ public class SparkReduceRecordHandler extends SparkRecordHandler { } int rowIdx = 0; +int batchBytes = 0; try { while (values.hasNext()) { /* deserialize value into columns */ @@ -381,11 +383,13 @@ public class SparkReduceRecordHandler extends SparkRecordHandler { VectorizedBatchUtil.addRowToBatchFrom(valueObj, valueStructInspectors[tag], rowIdx, keysColumnOffset, batch, buffer); +batchBytes += valueWritable.getLength(); rowIdx++; -if (rowIdx >= BATCH_SIZE) { +if (rowIdx >= BATCH_SIZE || batchBytes > BATCH_BYTES) { VectorizedBatchUtil.setBatchSize(batch, rowIdx); reducer.process(batch, tag); rowIdx = 0; + batchBytes = 0; if (isLogInfoEnabled) { logMemoryInfo(); } http://git-wip-us.apache.org/repos/asf/hive/blob/e434f832/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java index ad8b9e0..d9caa47 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java @@ -103,6 +103,8 @@ public class ReduceRecordSource implements RecordSource { // number of columns pertaining to keys in a vectorized row batch private int firstValueColumnOffset; + private final int BATCH_BYTES = VectorizedRowBatch.DEFAULT_BYTES; + private StructObjectInspector keyStructInspector; private StructObjectInspector valueStructInspectors; @@ -435,6 +437,7 @@ public class ReduceRecordSource implements RecordSource { final int maxSize = batch.getMaxSize(); Preconditions.checkState(maxSize > 0); int rowIdx = 0; +int batchBytes = keyBytes.length; try { for (Object value : values) { if (valueLazyBinaryDeserializeToRow != null) { @@ -442,6 +445,7 @@ public class ReduceRecordSource implements RecordSource { BytesWritable valueWritable = (BytesWritable) value; byte[] valueBytes = valueWritable.getBytes(); int valueLength = valueWritable.getLength(); + batchBytes += valueLength; // l4j.info("ReduceRecordSource processVectorGroup valueBytes " + valueLength + " " + // VectorizedBatchUtil.displayBytes(valueBytes, 0, valueLength)); @@ -450,7 +454,7 @@ public class ReduceRecordSource implements RecordSource { valueLazyBinaryDeserializeToRow.deserialize(batch, rowIdx); } rowIdx++; -if (rowIdx >= maxSize) { +if (rowIdx >= ma
hive git commit: HIVE-16182: Semijoin: Avoid VectorHashKeyWrapper allocations for the bloom hash aggregate (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master ba4f6e7b1 -> bfa035024 HIVE-16182: Semijoin: Avoid VectorHashKeyWrapper allocations for the bloom hash aggregate (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bfa03502 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bfa03502 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bfa03502 Branch: refs/heads/master Commit: bfa035024082a08c0fde12a3c58f2b59ec57ef99 Parents: ba4f6e7 Author: Gopal V Authored: Mon Mar 13 23:11:37 2017 +0530 Committer: Gopal V Committed: Mon Mar 13 23:11:37 2017 +0530 -- .../ql/exec/vector/VectorHashKeyWrapper.java| 37 +++- .../exec/vector/VectorHashKeyWrapperBatch.java | 2 +- 2 files changed, 37 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/bfa03502/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java index 2bd1850..5de59b1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java @@ -47,6 +47,8 @@ public class VectorHashKeyWrapper extends KeyWrapper { private static final Timestamp[] EMPTY_TIMESTAMP_ARRAY = new Timestamp[0]; private static final HiveIntervalDayTime[] EMPTY_INTERVAL_DAY_TIME_ARRAY = new HiveIntervalDayTime[0]; + public static final VectorHashKeyWrapper EMPTY_KEY_WRAPPER = new EmptyVectorHashKeyWrapper(); + private long[] longValues; private double[] doubleValues; @@ -63,7 +65,7 @@ public class VectorHashKeyWrapper extends KeyWrapper { private boolean[] isNull; private int hashcode; - public VectorHashKeyWrapper(int longValuesCount, int doubleValuesCount, + private VectorHashKeyWrapper(int longValuesCount, int doubleValuesCount, int byteValuesCount, int decimalValuesCount, int timestampValuesCount, int intervalDayTimeValuesCount) { longValues = longValuesCount > 0 ? new long[longValuesCount] : EMPTY_LONG_ARRAY; @@ -97,6 +99,17 @@ public class VectorHashKeyWrapper extends KeyWrapper { private VectorHashKeyWrapper() { } + public static VectorHashKeyWrapper allocate(int longValuesCount, int doubleValuesCount, + int byteValuesCount, int decimalValuesCount, int timestampValuesCount, + int intervalDayTimeValuesCount) { +if ((longValuesCount + doubleValuesCount + byteValuesCount + decimalValuesCount ++ timestampValuesCount + intervalDayTimeValuesCount) == 0) { + return EMPTY_KEY_WRAPPER; +} +return new VectorHashKeyWrapper(longValuesCount, doubleValuesCount, byteValuesCount, +decimalValuesCount, timestampValuesCount, intervalDayTimeValuesCount); + } + @Override public void getNewKey(Object row, ObjectInspector rowInspector) throws HiveException { throw new HiveException("Should not be called"); @@ -415,5 +428,27 @@ public class VectorHashKeyWrapper extends KeyWrapper { public HiveIntervalDayTime getIntervalDayTime(int i) { return intervalDayTimeValues[i]; } + + public static final class EmptyVectorHashKeyWrapper extends VectorHashKeyWrapper { +private EmptyVectorHashKeyWrapper() { + super(0, 0, 0, 0, 0, 0); + // no need to override assigns - all assign ops will fail due to 0 size +} + +@Override +protected Object clone() { + // immutable + return this; +} + +@Override +public boolean equals(Object that) { + if (that == this) { +// should only be one object +return true; + } + return super.equals(that); +} + } } http://git-wip-us.apache.org/repos/asf/hive/blob/bfa03502/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java index b4708b5..f68228c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java @@ -744,7 +744,7 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo { } public VectorHashKeyWrapper allocateKeyWrapper() { -return new VectorHashKeyWrapper(longIndices.length, doubleIndices.length, +return VectorHashKeyWrapper.allocate(longIndices.length, doubleIndices.length,
hive git commit: HIVE-16050: Regression: Union of null with non-null (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master 12b27a355 -> a6366fc07 HIVE-16050: Regression: Union of null with non-null (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a6366fc0 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a6366fc0 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a6366fc0 Branch: refs/heads/master Commit: a6366fc07931f374af77d0ab208404027bf165a3 Parents: 12b27a3 Author: Gopal V Authored: Wed Mar 1 17:57:54 2017 +0530 Committer: Gopal V Committed: Wed Mar 1 17:58:06 2017 +0530 -- .../org/apache/hadoop/hive/ql/exec/FunctionRegistry.java | 9 + ql/src/test/queries/clientpositive/union_null.q | 3 +++ ql/src/test/results/clientpositive/spark/union_null.q.out | 10 ++ ql/src/test/results/clientpositive/union_null.q.out | 10 ++ 4 files changed, 32 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a6366fc0/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java -- 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 0f05160..aaf2399 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 @@ -728,6 +728,15 @@ public final class FunctionRegistry { PrimitiveGrouping pgA = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(pcA); PrimitiveGrouping pgB = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(pcB); + +// untyped nulls +if (pgA == PrimitiveGrouping.VOID_GROUP) { + return b; +} +if (pgB == PrimitiveGrouping.VOID_GROUP) { + return a; +} + if (pgA != pgB) { return null; } http://git-wip-us.apache.org/repos/asf/hive/blob/a6366fc0/ql/src/test/queries/clientpositive/union_null.q -- diff --git a/ql/src/test/queries/clientpositive/union_null.q b/ql/src/test/queries/clientpositive/union_null.q index 45448b4..aacf310 100644 --- a/ql/src/test/queries/clientpositive/union_null.q +++ b/ql/src/test/queries/clientpositive/union_null.q @@ -8,3 +8,6 @@ select x from (select * from (select value as x from src order by x limit 5)a un set hive.cbo.returnpath.hiveop=false; -- HIVE-4837 select * from (select * from (select cast(null as string) as N from src1 group by key)a UNION ALL select * from (select cast(null as string) as N from src1 group by key)b ) a; + +-- HIVE-16050 +select null as c1 UNION ALL select 1 as c1; http://git-wip-us.apache.org/repos/asf/hive/blob/a6366fc0/ql/src/test/results/clientpositive/spark/union_null.q.out -- diff --git a/ql/src/test/results/clientpositive/spark/union_null.q.out b/ql/src/test/results/clientpositive/spark/union_null.q.out index e196ff3..00bd9d9 100644 --- a/ql/src/test/results/clientpositive/spark/union_null.q.out +++ b/ql/src/test/results/clientpositive/spark/union_null.q.out @@ -74,3 +74,13 @@ NULL NULL NULL NULL +PREHOOK: query: select null as c1 UNION ALL select 1 as c1 +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table + A masked pattern was here +POSTHOOK: query: select null as c1 UNION ALL select 1 as c1 +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table + A masked pattern was here +NULL +1 http://git-wip-us.apache.org/repos/asf/hive/blob/a6366fc0/ql/src/test/results/clientpositive/union_null.q.out -- diff --git a/ql/src/test/results/clientpositive/union_null.q.out b/ql/src/test/results/clientpositive/union_null.q.out index e196ff3..00bd9d9 100644 --- a/ql/src/test/results/clientpositive/union_null.q.out +++ b/ql/src/test/results/clientpositive/union_null.q.out @@ -74,3 +74,13 @@ NULL NULL NULL NULL +PREHOOK: query: select null as c1 UNION ALL select 1 as c1 +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table + A masked pattern was here +POSTHOOK: query: select null as c1 UNION ALL select 1 as c1 +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table + A masked pattern was here +NULL +1
hive git commit: HIVE-14573: Vectorization: Implement StringExpr::find() (Teddy Choi, reviewed by Gopal V)
Repository: hive Updated Branches: refs/heads/master ea9e851de -> fe9a6d538 HIVE-14573: Vectorization: Implement StringExpr::find() (Teddy Choi, reviewed by Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fe9a6d53 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fe9a6d53 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fe9a6d53 Branch: refs/heads/master Commit: fe9a6d5387154cec1c3fa5d82fda570103155656 Parents: ea9e851 Author: Teddy Choi Authored: Mon Feb 6 17:00:52 2017 -0800 Committer: Gopal V Committed: Mon Feb 6 17:02:08 2017 -0800 -- .../vectorization/AbstractExpression.java | 20 +- .../vectorization/VectorizedLikeBench.java | 67 ...AbstractFilterStringColLikeStringScalar.java | 25 ++-- .../FilterStringColLikeStringScalar.java| 11 +--- .../ql/exec/vector/expressions/StringExpr.java | 60 ++ .../exec/vector/expressions/TestStringExpr.java | 60 ++ 6 files changed, 214 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fe9a6d53/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java -- diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java index 94af3e0..879b437 100644 --- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java +++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/AbstractExpression.java @@ -13,6 +13,7 @@ */ package org.apache.hive.benchmark.vectorization; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; @@ -35,7 +36,7 @@ import java.util.concurrent.TimeUnit; @BenchmarkMode(Mode.AverageTime) @Fork(1) @State(Scope.Thread) -@OutputTimeUnit(TimeUnit.NANOSECONDS) +@OutputTimeUnit(TimeUnit.MILLISECONDS) public abstract class AbstractExpression { private static final int DEFAULT_ITER_TIME = 100; protected VectorExpression expression; @@ -59,6 +60,9 @@ public abstract class AbstractExpression { @Measurement(iterations = 2, time = 2, timeUnit = TimeUnit.MILLISECONDS) public void bench() { for (int i = 0; i < DEFAULT_ITER_TIME; i++) { + rowBatch.selectedInUse = false; + rowBatch.size = VectorizedRowBatch.DEFAULT_SIZE; + expression.evaluate(rowBatch); } } @@ -147,4 +151,18 @@ public abstract class AbstractExpression { return columnVector; } + protected BytesColumnVector getBytesColumnVector() { +BytesColumnVector columnVector = new BytesColumnVector(VectorizedRowBatch.DEFAULT_SIZE); +Random random = new Random(); +int length = 16; +for (int i = 0; i != VectorizedRowBatch.DEFAULT_SIZE; i++) { + columnVector.vector[i] = new byte[length]; + columnVector.start[i] = 0; + columnVector.length[i] = length; + for (int j = 0; j < length; j++) { +columnVector.vector[i][j] = (byte)(random.nextInt(+ 'c' - 'a' + 1) + 'a'); + } +} +return columnVector; + } } http://git-wip-us.apache.org/repos/asf/hive/blob/fe9a6d53/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLikeBench.java -- diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLikeBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLikeBench.java new file mode 100644 index 000..136c01b --- /dev/null +++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLikeBench.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 langu
hive git commit: HIVE-15655: Optimizer: Allow config option to disable n-way JOIN merging (Gopal V, reviewed by Jesus Camacho Rodriguez)
Repository: hive Updated Branches: refs/heads/master c31c2963d -> 422b35759 HIVE-15655: Optimizer: Allow config option to disable n-way JOIN merging (Gopal V, reviewed by Jesus Camacho Rodriguez) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/422b3575 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/422b3575 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/422b3575 Branch: refs/heads/master Commit: 422b3575905ceb725a413ef169dfd1035a3b0de3 Parents: c31c296 Author: Gopal V Authored: Wed Jan 25 00:14:39 2017 -0800 Committer: Gopal V Committed: Wed Jan 25 00:14:50 2017 -0800 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 2 + .../test/resources/testconfiguration.properties | 1 + .../hadoop/hive/ql/parse/CalcitePlanner.java| 4 +- .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 5 +- .../test/queries/clientpositive/tez_nway_join.q | 29 ++ .../clientpositive/llap/tez_nway_join.q.out | 501 +++ 6 files changed, 539 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/422b3575/common/src/java/org/apache/hadoop/hive/conf/HiveConf.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 2ccb764..291ca7d 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3247,6 +3247,8 @@ public class HiveConf extends Configuration { "Maximum total data size in dynamic pruning."), NWAYJOINREORDER("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", true, + "Merge adjacent joins into a single n-way join"), HIVE_LOG_N_RECORDS("hive.log.every.n.records", 0L, new RangeValidator(0L, null), "If value is greater than 0 logs in fixed intervals of size n rather than exponentially."), HIVE_MSCK_PATH_VALIDATION("hive.msck.path.validation", "throw", http://git-wip-us.apache.org/repos/asf/hive/blob/422b3575/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index bd76b7d..e06e643 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -581,6 +581,7 @@ minillaplocal.query.files=acid_globallimit.q,\ tez_join_tests.q,\ tez_joins_explain.q,\ tez_multi_union.q,\ + tez_nway_join.q,\ tez_schema_evolution.q,\ tez_self_join.q,\ tez_smb_1.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/422b3575/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java -- 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 fd99ba6..e9bf3e4 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 @@ -385,7 +385,7 @@ public class CalcitePlanner extends SemanticAnalyzer { // PartitionList is not evaluated until the run phase. getMetaData(getQB()); -disableJoinMerge = false; +disableJoinMerge = defaultJoinMerge; sinkOp = genPlan(getQB()); LOG.info("CBO Succeeded; optimized logical plan."); this.ctx.setCboInfo("Plan optimized by CBO."); @@ -436,7 +436,7 @@ public class CalcitePlanner extends SemanticAnalyzer { } } finally { runCBO = false; - disableJoinMerge = false; + disableJoinMerge = defaultJoinMerge; disableSemJoinReordering = false; if (reAnalyzeAST) { init(true); http://git-wip-us.apache.org/repos/asf/hive/blob/422b3575/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java -- 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 e5d0101..6249475 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 @@ -318,6 +318,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { protected boolean partialscan; protected volatile boolean disableJoinM
hive git commit: HIVE-15420: LLAP UI: Relativize resources to allow proxied/secured views (Gopal V, reviewed by Rajesh Balamohan)
Repository: hive Updated Branches: refs/heads/master 382dc2084 -> 7f46c8d91 HIVE-15420: LLAP UI: Relativize resources to allow proxied/secured views (Gopal V, reviewed by Rajesh Balamohan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7f46c8d9 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7f46c8d9 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7f46c8d9 Branch: refs/heads/master Commit: 7f46c8d9182baca91557cc4322c859dedef88315 Parents: 382dc20 Author: Gopal V Authored: Fri Dec 16 13:00:33 2016 -0800 Committer: Gopal V Committed: Fri Dec 16 13:00:33 2016 -0800 -- .../src/main/resources/hive-webapps/llap/index.html | 14 +++--- .../main/resources/hive-webapps/llap/js/metrics.js| 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7f46c8d9/llap-server/src/main/resources/hive-webapps/llap/index.html -- diff --git a/llap-server/src/main/resources/hive-webapps/llap/index.html b/llap-server/src/main/resources/hive-webapps/llap/index.html index 31a27a0..73c03c1 100644 --- a/llap-server/src/main/resources/hive-webapps/llap/index.html +++ b/llap-server/src/main/resources/hive-webapps/llap/index.html @@ -25,19 +25,19 @@ - - - - - - + + + + + + - + http://git-wip-us.apache.org/repos/asf/hive/blob/7f46c8d9/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js -- diff --git a/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js b/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js index 7bb2890..4c41c34 100644 --- a/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js +++ b/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js @@ -227,7 +227,7 @@ $(function() { var views = [llap.view.Hostname, llap.view.Heap, llap.view.Cache, llap.view.Executors, llap.view.Tasks, llap.view.System] setInterval(function() { -$.getJSON("/jmx", function(jmx){ +$.getJSON("jmx", function(jmx){ models.forEach(function (m) { m.push(jmx); }); views.forEach(function (v) { v.refresh(); }); });
hive git commit: HIVE-15196: LLAP UI: HIVE-14984 broke LLAP UI (Barna Zsombor Klara via Gopal V)
Repository: hive Updated Branches: refs/heads/master 76f1f41f0 -> e1c1b062a HIVE-15196: LLAP UI: HIVE-14984 broke LLAP UI (Barna Zsombor Klara via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e1c1b062 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e1c1b062 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e1c1b062 Branch: refs/heads/master Commit: e1c1b062ac766acad990c7e44d8fcf5d6af35f86 Parents: 76f1f41 Author: Barna Zsombor Klara Authored: Wed Nov 23 02:38:02 2016 -0800 Committer: Gopal V Committed: Wed Nov 23 02:38:44 2016 -0800 -- .../java/org/apache/hive/http/HttpServer.java | 8 ++- .../services/impl/TestLlapWebServices.java | 69 .../apache/hive/service/server/HiveServer2.java | 1 + .../hive/service/server/TestHS2HttpServer.java | 24 +++ 4 files changed, 89 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e1c1b062/common/src/java/org/apache/hive/http/HttpServer.java -- diff --git a/common/src/java/org/apache/hive/http/HttpServer.java b/common/src/java/org/apache/hive/http/HttpServer.java index 42d2959..f99f18a 100644 --- a/common/src/java/org/apache/hive/http/HttpServer.java +++ b/common/src/java/org/apache/hive/http/HttpServer.java @@ -120,6 +120,7 @@ public class HttpServer { private String spnegoKeytab; private boolean useSPNEGO; private boolean useSSL; +private String contextRootRewriteTarget = "/index.html"; private final List>> servlets = new LinkedList>>(); @@ -197,6 +198,11 @@ public class HttpServer { return this; } +public Builder setContextRootRewriteTarget(String contextRootRewriteTarget) { + this.contextRootRewriteTarget = contextRootRewriteTarget; + return this; +} + public Builder addServlet(String endpoint, Class servlet) { servlets.add(new Pair>(endpoint, servlet)); return this; @@ -394,7 +400,7 @@ public class HttpServer { RewriteRegexRule rootRule = new RewriteRegexRule(); rootRule.setRegex("^/$"); -rootRule.setReplacement("/hiveserver2.jsp"); +rootRule.setReplacement(b.contextRootRewriteTarget); rootRule.setTerminating(true); rwHandler.addRule(rootRule); http://git-wip-us.apache.org/repos/asf/hive/blob/e1c1b062/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/services/impl/TestLlapWebServices.java -- diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/services/impl/TestLlapWebServices.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/services/impl/TestLlapWebServices.java new file mode 100644 index 000..833a8f5 --- /dev/null +++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/services/impl/TestLlapWebServices.java @@ -0,0 +1,69 @@ +/* + * Licensed 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.llap.daemon.services.impl; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.io.StringWriter; +import java.net.HttpURLConnection; +import java.net.URL; + +public class TestLlapWebServices { + + private static LlapWebServices llapWS = null; + private static int llapWSPort; + + @BeforeClass + public static void beforeTests() throws Exception { +llapWSPort = MetaStoreUtils.findFreePortExcepting( + Integer.valueOf(HiveConf.ConfVars.LLAP_DAEMON_WEB_PORT.getDefaultValue())); +llapWS = new LlapWebServices(llapWSPort, null, null); +llapWS.init(new HiveConf()); +llapWS.start(); +Thread.sleep(5000); + } + + @Test + public void testContextRootUrlRewrite() throws Exception { +String contextRootURL = "http://localhost:"; + llapWSPort + "/"; +String contextRootContent = getURLResponseAsString(contextRootURL); + +String indexHtmlUrl = "http://localhost:"; + llapWSPort + "/index.html"; +String indexHtmlContent = getURLResponseAsS
hive git commit: HIVE-15125: LLAP: Parallelize slider package generator (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master a4a00b20c -> aa7c9cd61 HIVE-15125: LLAP: Parallelize slider package generator (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/aa7c9cd6 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/aa7c9cd6 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/aa7c9cd6 Branch: refs/heads/master Commit: aa7c9cd614804c0bf683745614f7a2b264ce72bf Parents: a4a00b2 Author: Gopal V Authored: Tue Nov 15 14:52:26 2016 -0800 Committer: Gopal V Committed: Tue Nov 15 14:52:26 2016 -0800 -- .../hadoop/hive/llap/cli/LlapServiceDriver.java | 685 +++ 1 file changed, 390 insertions(+), 295 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/aa7c9cd6/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java -- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java index 6f533df..dfd2f7b 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java @@ -34,6 +34,13 @@ import java.util.Collection; import java.util.List; import java.util.Properties; import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration; @@ -71,6 +78,7 @@ import org.eclipse.jetty.server.ssl.SslSocketConnector; import org.json.JSONObject; import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; public class LlapServiceDriver { @@ -153,11 +161,21 @@ public class LlapServiceDriver { } } + private static abstract class NamedCallable implements Callable { +public final String taskName; +public NamedCallable (String name) { + this.taskName = name; +} +public String getName() { + return taskName; +} + } + private void run(String[] args) throws Exception { LlapOptionsProcessor optionsProcessor = new LlapOptionsProcessor(); -LlapOptions options = optionsProcessor.processOptions(args); +final LlapOptions options = optionsProcessor.processOptions(args); -Properties propsDirectOptions = new Properties(); +final Properties propsDirectOptions = new Properties(); if (options == null) { // help @@ -171,346 +189,418 @@ public class LlapServiceDriver { throw new Exception("Cannot load any configuration to run command"); } -FileSystem fs = FileSystem.get(conf); -FileSystem lfs = FileSystem.getLocal(conf).getRawFileSystem(); - -// needed so that the file is actually loaded into configuration. -for (String f : NEEDED_CONFIGS) { - conf.addResource(f); - if (conf.getResource(f) == null) { -throw new Exception("Unable to find required config file: " + f); - } -} -for (String f : OPTIONAL_CONFIGS) { - conf.addResource(f); -} - -conf.reloadConfiguration(); +final long t0 = System.nanoTime(); -populateConfWithLlapProperties(conf, options.getConfig()); +final FileSystem fs = FileSystem.get(conf); +final FileSystem lfs = FileSystem.getLocal(conf).getRawFileSystem(); +final ExecutorService executor = + Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() / 2, +new ThreadFactoryBuilder().setNameFormat("llap-pkg-%d").build()); +final CompletionService asyncRunner = new ExecutorCompletionService(executor); -if (options.getName() != null) { - // update service registry configs - caveat: this has nothing to do with the actual settings - // as read by the AM - // if needed, use --hiveconf llap.daemon.service.hosts=@llap0 to dynamically switch between - // instances - conf.set(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname, "@" + options.getName()); - propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname, - "@" + options.getName()); -} - -if (options.getLogger() != null) { - HiveConf.setVar(conf, ConfVars.LLAP_DAEMON_LOGGER, options.getLogger()); - propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_LOGGER.varname, options.getLogger()); -} +try { -if (options.getSize() != -1) { - if (options.ge
hive git commit: HIVE-15136: (addendum)
Repository: hive Updated Branches: refs/heads/master be4fbeba0 -> 52ba014fc HIVE-15136: (addendum) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/52ba014f Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/52ba014f Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/52ba014f Branch: refs/heads/master Commit: 52ba014fc0bca87a5d10b7d15ab2f6bf2a101cd5 Parents: be4fbeb Author: Gopal V Authored: Fri Nov 11 18:24:17 2016 -0800 Committer: Gopal V Committed: Fri Nov 11 18:24:17 2016 -0800 -- llap-server/src/main/resources/templates.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/52ba014f/llap-server/src/main/resources/templates.py -- diff --git a/llap-server/src/main/resources/templates.py b/llap-server/src/main/resources/templates.py index 0176384..5684f33 100644 --- a/llap-server/src/main/resources/templates.py +++ b/llap-server/src/main/resources/templates.py @@ -114,7 +114,7 @@ resources = """ "yarn.role.priority": "1", "yarn.component.instances": "%(instances)d", "yarn.memory": "%(container.mb)d", - "yarn.component.placement.policy" : "%(placement)" + "yarn.component.placement.policy" : "%(placement)d" } } }
hive git commit: HIVE-15123: LLAP UI: The UI should work even if the cache is disabled (Gopal V, reviewed by Siddharth Seth)
Repository: hive Updated Branches: refs/heads/branch-2.1 662968b40 -> 1537ccb37 HIVE-15123: LLAP UI: The UI should work even if the cache is disabled (Gopal V, reviewed by Siddharth Seth) (cherry picked from commit ed8cc8aa807ed4cc52b20eaf1274b52d530bba3a) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1537ccb3 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1537ccb3 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1537ccb3 Branch: refs/heads/branch-2.1 Commit: 1537ccb374f8ef469e7780252d6ba15f946bb4bc Parents: 662968b Author: Gopal V Authored: Fri Nov 4 16:04:15 2016 -0700 Committer: Gopal V Committed: Fri Nov 4 16:05:42 2016 -0700 -- .../resources/hive-webapps/llap/js/metrics.js | 26 +--- 1 file changed, 17 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1537ccb3/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js -- diff --git a/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js b/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js index b67a6e3..7bb2890 100644 --- a/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js +++ b/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js @@ -79,11 +79,19 @@ llap.model.LlapDaemonCacheMetrics = new function() { this.fill_rate = trendlist(50); this.push = function(jmx) { var bean = jmxbean(jmx, this.name); - this.cache_max = bean["CacheCapacityTotal"]/(1024*1024); - this.cache_used = bean["CacheCapacityUsed"]/(1024*1024); - this.cache_reqs = bean["CacheReadRequests"]; - this.fill_rate.add((this.cache_used*100.0)/this.cache_max); - this.hit_rate.add(bean["CacheHitRatio"]*100.0); + if (bean) { +this.cache_max = bean["CacheCapacityTotal"]/(1024*1024); +this.cache_used = bean["CacheCapacityUsed"]/(1024*1024); +this.cache_reqs = bean["CacheReadRequests"]; +this.fill_rate.add((this.cache_used*100.0)/this.cache_max); +this.hit_rate.add(bean["CacheHitRatio"]*100.0); + } else { +this.cache_max = -1; +this.cache_used = -1; +this.cache_reqs = -1; +this.fill_rate.add(0); +this.hit_rate.add(-1); + } } return this; } @@ -106,10 +114,10 @@ llap.model.LlapDaemonExecutorMetrics = new function() { this.push = function(jmx) { var bean = jmxbean(jmx, this.name); this.queue_rate.add(bean["ExecutorNumQueuedRequests"] || 0); - this.lost_time = bean["PreemptionTimeLost"]; - this.num_tasks = bean["ExecutorTotalRequestsHandled"]; - this.interrupted_tasks = bean["ExecutorTotalInterrupted"]; - this.failed_tasks = bean["ExecutorTotalExecutionFailure"]; + this.lost_time = bean["PreemptionTimeLost"] || 0; + this.num_tasks = bean["ExecutorTotalRequestsHandled"]; + this.interrupted_tasks = bean["ExecutorTotalInterrupted"] || 0; + this.failed_tasks = bean["ExecutorTotalExecutionFailure"] || 0; } return this; }
hive git commit: HIVE-15123: LLAP UI: The UI should work even if the cache is disabled (Gopal V, reviewed by Siddharth Seth)
Repository: hive Updated Branches: refs/heads/master 97f7d7b18 -> ed8cc8aa8 HIVE-15123: LLAP UI: The UI should work even if the cache is disabled (Gopal V, reviewed by Siddharth Seth) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ed8cc8aa Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ed8cc8aa Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ed8cc8aa Branch: refs/heads/master Commit: ed8cc8aa807ed4cc52b20eaf1274b52d530bba3a Parents: 97f7d7b Author: Gopal V Authored: Fri Nov 4 16:04:15 2016 -0700 Committer: Gopal V Committed: Fri Nov 4 16:04:15 2016 -0700 -- .../resources/hive-webapps/llap/js/metrics.js | 26 +--- 1 file changed, 17 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ed8cc8aa/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js -- diff --git a/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js b/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js index b67a6e3..7bb2890 100644 --- a/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js +++ b/llap-server/src/main/resources/hive-webapps/llap/js/metrics.js @@ -79,11 +79,19 @@ llap.model.LlapDaemonCacheMetrics = new function() { this.fill_rate = trendlist(50); this.push = function(jmx) { var bean = jmxbean(jmx, this.name); - this.cache_max = bean["CacheCapacityTotal"]/(1024*1024); - this.cache_used = bean["CacheCapacityUsed"]/(1024*1024); - this.cache_reqs = bean["CacheReadRequests"]; - this.fill_rate.add((this.cache_used*100.0)/this.cache_max); - this.hit_rate.add(bean["CacheHitRatio"]*100.0); + if (bean) { +this.cache_max = bean["CacheCapacityTotal"]/(1024*1024); +this.cache_used = bean["CacheCapacityUsed"]/(1024*1024); +this.cache_reqs = bean["CacheReadRequests"]; +this.fill_rate.add((this.cache_used*100.0)/this.cache_max); +this.hit_rate.add(bean["CacheHitRatio"]*100.0); + } else { +this.cache_max = -1; +this.cache_used = -1; +this.cache_reqs = -1; +this.fill_rate.add(0); +this.hit_rate.add(-1); + } } return this; } @@ -106,10 +114,10 @@ llap.model.LlapDaemonExecutorMetrics = new function() { this.push = function(jmx) { var bean = jmxbean(jmx, this.name); this.queue_rate.add(bean["ExecutorNumQueuedRequests"] || 0); - this.lost_time = bean["PreemptionTimeLost"]; - this.num_tasks = bean["ExecutorTotalRequestsHandled"]; - this.interrupted_tasks = bean["ExecutorTotalInterrupted"]; - this.failed_tasks = bean["ExecutorTotalExecutionFailure"]; + this.lost_time = bean["PreemptionTimeLost"] || 0; + this.num_tasks = bean["ExecutorTotalRequestsHandled"]; + this.interrupted_tasks = bean["ExecutorTotalInterrupted"] || 0; + this.failed_tasks = bean["ExecutorTotalExecutionFailure"] || 0; } return this; }
hive git commit: HIVE-14837: JDBC: standalone jar is missing hadoop core dependencies (Tao Li, via Gopal V)
Repository: hive Updated Branches: refs/heads/master 080de9708 -> 7968e1efe HIVE-14837: JDBC: standalone jar is missing hadoop core dependencies (Tao Li, via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7968e1ef Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7968e1ef Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7968e1ef Branch: refs/heads/master Commit: 7968e1efe9453dd8d3ca8a013d952d25ba9bb287 Parents: 080de97 Author: Tao Li Authored: Mon Oct 24 18:12:48 2016 -0700 Committer: Gopal V Committed: Mon Oct 24 18:13:36 2016 -0700 -- jdbc/pom.xml | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7968e1ef/jdbc/pom.xml -- diff --git a/jdbc/pom.xml b/jdbc/pom.xml index b29739b..5ae2266 100644 --- a/jdbc/pom.xml +++ b/jdbc/pom.xml @@ -180,7 +180,12 @@ org.apache.commons:commons-compress commons-configuration:commons-configuration - org.apache.hadoop:* + org.apache.hadoop:hadoop-yarn* + org.apache.hadoop:hadoop-mapreduce* + org.apache.hadoop:hadoop-hdfs + org.apache.hadoop:hadoop-client + org.apache.hadoop:hadoop-annotations + org.apache.hadoop:hadoop-auth org.apache.hive:hive-ant org.apache.ant:* junit:*
hive git commit: HIVE-14618: beeline fetch logging delays before query completion (Tao Li, via Gopal V)
Repository: hive Updated Branches: refs/heads/master b63ad9b0d -> 349445ccd HIVE-14618: beeline fetch logging delays before query completion (Tao Li, via Gopal V) Signed-off-by: Gopal V Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/349445cc Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/349445cc Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/349445cc Branch: refs/heads/master Commit: 349445ccd8c5f0fc3669c71dcbba92ea0e40c92d Parents: b63ad9b Author: Tao Li Authored: Fri Sep 2 11:12:12 2016 +1000 Committer: Gopal V Committed: Fri Sep 2 11:12:20 2016 +1000 -- .../src/java/org/apache/hive/service/cli/CLIService.java | 11 --- .../test/org/apache/hive/service/cli/CLIServiceTest.java | 10 +++--- 2 files changed, 15 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/349445cc/service/src/java/org/apache/hive/service/cli/CLIService.java -- 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 ed52b4a..662e55c 100644 --- a/service/src/java/org/apache/hive/service/cli/CLIService.java +++ b/service/src/java/org/apache/hive/service/cli/CLIService.java @@ -425,14 +425,19 @@ public class CLIService extends CompositeService implements ICLIService { Operation operation = sessionManager.getOperationManager().getOperation(opHandle); /** * If this is a background operation run asynchronously, - * we block for a configured duration, before we return - * (duration: HIVE_SERVER2_LONG_POLLING_TIMEOUT). + * we block for a duration determined by a step function, before we return * However, if the background operation is complete, we return immediately. */ if (operation.shouldRunAsync()) { HiveConf conf = operation.getParentSession().getHiveConf(); - long timeout = HiveConf.getTimeVar(conf, + long maxTimeout = HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT, TimeUnit.MILLISECONDS); + + final long elapsed = System.currentTimeMillis() - operation.getBeginTime(); + // A step function to increase the polling timeout by 500 ms every 10 sec, + // starting from 500 ms up to HIVE_SERVER2_LONG_POLLING_TIMEOUT + final long timeout = Math.min(maxTimeout, (elapsed / TimeUnit.SECONDS.toMillis(10) + 1) * 500); + try { operation.getBackgroundHandle().get(timeout, TimeUnit.MILLISECONDS); } catch (TimeoutException e) { http://git-wip-us.apache.org/repos/asf/hive/blob/349445cc/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java -- diff --git a/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java b/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java index 17d45ec..237fcc0 100644 --- a/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java +++ b/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java @@ -528,14 +528,15 @@ public abstract class CLIServiceTest { } private OperationStatus waitForAsyncQuery(OperationHandle opHandle, - OperationState expectedState, long longPollingTimeout) throws HiveSQLException { + OperationState expectedState, long maxLongPollingTimeout) throws HiveSQLException { long testIterationTimeout = System.currentTimeMillis() + 10; long longPollingStart; long longPollingEnd; long longPollingTimeDelta; OperationStatus opStatus = null; OperationState state = null; -int count = 0; +int count = 0; +long start = System.currentTimeMillis(); while (true) { // Break if iteration times out if (System.currentTimeMillis() > testIterationTimeout) { @@ -560,8 +561,11 @@ public abstract class CLIServiceTest { } else { // Verify that getOperationStatus returned only after the long polling timeout longPollingTimeDelta = longPollingEnd - longPollingStart; +// Calculate the expected timeout based on the elapsed time between waiting start time and polling start time +long elapsed = longPollingStart - start; +long expectedTimeout = Math.min(maxLongPollingTimeout, (elapsed / TimeUnit.SECONDS.toMillis(10) + 1) * 500); // Scale down by a factor of 0.9 to account for approximate values -assertTrue(longPollingTimeDelta - 0.9*longPollingTimeout > 0); +assertTrue(longPollingTimeDelta - 0.9*expectedTimeout > 0); } } assertEquals(expectedState, client.getOperationStatus(opHandle).getState());
hive git commit: HIVE-14648: LLAP: Avoid private pages in the SSD cache (Gopal V, reviewed by Sergey Shelukhin)
Repository: hive Updated Branches: refs/heads/master d35ad0677 -> 9a90c65d7 HIVE-14648: LLAP: Avoid private pages in the SSD cache (Gopal V, reviewed by Sergey Shelukhin) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9a90c65d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9a90c65d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9a90c65d Branch: refs/heads/master Commit: 9a90c65d70b8eb68bb575928ef233d57e589a7b0 Parents: d35ad06 Author: Gopal V Authored: Sat Aug 27 01:27:24 2016 -0700 Committer: Gopal V Committed: Sat Aug 27 01:27:24 2016 -0700 -- .../java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/9a90c65d/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java -- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java index b03927a..0c53779 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java @@ -340,7 +340,9 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca File rf = File.createTempFile("arena-", ".cache", cacheDir.toFile()); RandomAccessFile rwf = new RandomAccessFile(rf, "rw"); rwf.setLength(arenaSize); // truncate (TODO: posix_fallocate?) -ByteBuffer rwbuf = rwf.getChannel().map(MapMode.PRIVATE, 0, arenaSize); +// Use RW, not PRIVATE because the copy-on-write is irrelevant for a deleted file +// see discussion in YARN-5551 for the memory accounting discussion +ByteBuffer rwbuf = rwf.getChannel().map(MapMode.READ_WRITE, 0, arenaSize); // A mapping, once established, is not dependent upon the file channel that was used to // create it. delete file and hold onto the map rwf.close();
hive git commit: HIVE-14155: Vectorization: Custom UDF Vectorization annotations are ignored (Gopal V, reviewed by Ashutosh Chauhan)
Repository: hive Updated Branches: refs/heads/master 093341624 -> d35ad0677 HIVE-14155: Vectorization: Custom UDF Vectorization annotations are ignored (Gopal V, reviewed by Ashutosh Chauhan) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d35ad067 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d35ad067 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d35ad067 Branch: refs/heads/master Commit: d35ad06779650c8b5f6c259413bf03e9909ba72f Parents: 0933416 Author: Gopal V Authored: Sat Aug 27 01:25:28 2016 -0700 Committer: Gopal V Committed: Sat Aug 27 01:25:36 2016 -0700 -- itests/custom-udfs/pom.xml | 1 + .../udf-vectorized-badexample/pom.xml | 43 .../hive/it/custom/udfs/GenericUDFRot13.java| 32 ++ .../custom/udfs/vector/VectorStringRot13.java | 46 .../ql/exec/vector/VectorizationContext.java| 106 ++- .../test/queries/clientpositive/vector_udf3.q | 13 +++ .../results/clientpositive/vector_udf3.q.out| 76 + 7 files changed, 266 insertions(+), 51 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d35ad067/itests/custom-udfs/pom.xml -- diff --git a/itests/custom-udfs/pom.xml b/itests/custom-udfs/pom.xml index 3e7443c..b230b41 100644 --- a/itests/custom-udfs/pom.xml +++ b/itests/custom-udfs/pom.xml @@ -42,6 +42,7 @@ limitations under the License. udf-classloader-util udf-classloader-udf1 udf-classloader-udf2 +udf-vectorized-badexample http://git-wip-us.apache.org/repos/asf/hive/blob/d35ad067/itests/custom-udfs/udf-vectorized-badexample/pom.xml -- diff --git a/itests/custom-udfs/udf-vectorized-badexample/pom.xml b/itests/custom-udfs/udf-vectorized-badexample/pom.xml new file mode 100644 index 000..35c1a2f --- /dev/null +++ b/itests/custom-udfs/udf-vectorized-badexample/pom.xml @@ -0,0 +1,43 @@ + + +http://maven.apache.org/POM/4.0.0"; + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";> + 4.0.0 + +org.apache.hive +hive-it-custom-udfs +2.2.0-SNAPSHOT +../pom.xml + + + org.apache.hive.hive-it-custom-udfs + udf-vectorized-badexample + jar + Hive Integration - Custom UDFs - udf-vectorized-badexample + + + + org.apache.hive.hive-it-custom-udfs + udf-classloader-util + ${project.version} + + + + +../../.. + + + http://git-wip-us.apache.org/repos/asf/hive/blob/d35ad067/itests/custom-udfs/udf-vectorized-badexample/src/main/java/hive/it/custom/udfs/GenericUDFRot13.java -- diff --git a/itests/custom-udfs/udf-vectorized-badexample/src/main/java/hive/it/custom/udfs/GenericUDFRot13.java b/itests/custom-udfs/udf-vectorized-badexample/src/main/java/hive/it/custom/udfs/GenericUDFRot13.java new file mode 100644 index 000..8941175 --- /dev/null +++ b/itests/custom-udfs/udf-vectorized-badexample/src/main/java/hive/it/custom/udfs/GenericUDFRot13.java @@ -0,0 +1,32 @@ +package hive.it.custom.udfs; + +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.Text; +import hive.it.custom.udfs.vector.VectorStringRot13; + +@VectorizedExpressions(value = { VectorStringRot13.class }) +public class GenericUDFRot13 extends GenericUDF { + + @Override + public Object evaluate(DeferredObject[] arg0) throws HiveException { +/* this is the bad part - the vectorized UDF returns the right result */ +return new Text("Unvectorized"); + } + + @Override + public String getDisplayString(String[] arg0) { +return String.format("Rot13(%s)", arg0[0]); + } + + @Override + public ObjectInspector initialize(ObjectInspector[] arg0) + throws UDFArgumentException { +return PrimitiveObjectInspectorFactory.writableStringObjectInspector; + } + +} http://git-wip-us.apache.org/repos/asf/hive/blob/d35ad067/itests/custom-udfs/udf-vectorized-badexample/src/main/java/hive/it/custom/udfs/vector/VectorStringRot13.java -- diff --git a/itests/custom-udfs/udf-vectorized-badexample/src/main/java/hive/it/custom/udfs/vector/VectorSt
hive git commit: HIVE-14437: Vectorization: Optimize key misses in VectorMapJoinFastBytesHashTable (Gopal V, reviewed by Matt McCline)
Repository: hive Updated Branches: refs/heads/master bde285d33 -> e5dd6fdc2 HIVE-14437: Vectorization: Optimize key misses in VectorMapJoinFastBytesHashTable (Gopal V, reviewed by Matt McCline) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e5dd6fdc Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e5dd6fdc Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e5dd6fdc Branch: refs/heads/master Commit: e5dd6fdc21fa46a907dd9ca2af484cb7c18f258e Parents: bde285d Author: Gopal V Authored: Thu Aug 25 21:18:41 2016 -0700 Committer: Gopal V Committed: Thu Aug 25 21:18:41 2016 -0700 -- .../vector/mapjoin/fast/VectorMapJoinFastBytesHashTable.java| 5 - 1 file changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e5dd6fdc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastBytesHashTable.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastBytesHashTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastBytesHashTable.java index 7987723..dc0476b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastBytesHashTable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastBytesHashTable.java @@ -175,7 +175,10 @@ public abstract class VectorMapJoinFastBytesHashTable while (true) { int tripleIndex = slot * 3; // LOG.debug("VectorMapJoinFastBytesHashMap findReadSlot slot keyRefWord " + Long.toHexString(slotTriples[tripleIndex]) + " hashCode " + Long.toHexString(hashCode) + " entry hashCode " + Long.toHexString(slotTriples[tripleIndex + 1]) + " valueRefWord " + Long.toHexString(slotTriples[tripleIndex + 2])); - if (slotTriples[tripleIndex] != 0 && hashCode == slotTriples[tripleIndex + 1]) { + if (slotTriples[tripleIndex] == 0) { +// Given that we do not delete, an empty slot means no match. +return -1; + } else if (hashCode == slotTriples[tripleIndex + 1]) { // Finally, verify the key bytes match. if (keyStore.equalKey(slotTriples[tripleIndex], keyBytes, keyStart, keyLength, readPos)) {