hive git commit: HIVE-20512: Improve record and memory usage logging in SparkRecordHandler (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master ccbc5c383 -> d813b487a HIVE-20512: Improve record and memory usage logging in SparkRecordHandler (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d813b487 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d813b487 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d813b487 Branch: refs/heads/master Commit: d813b487ae4c04661a239a3a6b8a9bae12cf Parents: ccbc5c3 Author: Bharathkrishna Guruvayoor Murali Authored: Tue Nov 13 20:02:53 2018 -0600 Committer: Sahil Takiar Committed: Tue Nov 13 20:02:53 2018 -0600 -- .../ql/exec/spark/SparkMapRecordHandler.java| 11 +- .../exec/spark/SparkMergeFileRecordHandler.java | 2 + .../hive/ql/exec/spark/SparkRecordHandler.java | 109 +-- .../ql/exec/spark/SparkReduceRecordHandler.java | 14 +-- 4 files changed, 83 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d813b487/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java index 88dd12c..530131f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java @@ -140,9 +140,8 @@ public class SparkMapRecordHandler extends SparkRecordHandler { // Since there is no concept of a group, we don't invoke // startGroup/endGroup for a mapper mo.process((Writable) value); - if (LOG.isInfoEnabled()) { -logMemoryInfo(); - } + incrementRowNumber(); + } catch (Throwable e) { abort = true; Utilities.setMapWork(jc, null); @@ -164,11 +163,11 @@ public class SparkMapRecordHandler extends SparkRecordHandler { @Override public void close() { +super.close(); // No row was processed if (!anyRow) { LOG.trace("Close called. no row processed by map."); } - // check if there are IOExceptions if (!abort) { abort = execContext.getIoCxt().getIOExceptions(); @@ -188,10 +187,6 @@ public class SparkMapRecordHandler extends SparkRecordHandler { } } - if (LOG.isInfoEnabled()) { -logCloseInfo(); - } - ReportStats rps = new ReportStats(rp, jc); mo.preorderMap(rps); return; http://git-wip-us.apache.org/repos/asf/hive/blob/d813b487/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java index 8880bb6..409e3cc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java @@ -92,6 +92,7 @@ public class SparkMergeFileRecordHandler extends SparkRecordHandler { public void processRow(Object key, Object value) throws IOException { row[0] = key; row[1] = value; +incrementRowNumber(); try { mergeOp.process(row, 0); } catch (HiveException e) { @@ -108,6 +109,7 @@ public class SparkMergeFileRecordHandler extends SparkRecordHandler { @Override public void close() { +super.close(); LOG.info("Closing Merge Operator " + mergeOp.getName()); try { mergeOp.closeOp(abort); http://git-wip-us.apache.org/repos/asf/hive/blob/d813b487/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java index cb5bd7a..f7ea212 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec.spark; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.ql.exec.MapredContext; @@ -33,6 +34,10 @@ import java.lang.management.MemoryMXBean; import java.net.URLClassLoader; import java.util.Arrays; import java.util.Iterator; +import java.util.concurrent.ScheduledFuture; +import
hive git commit: HIVE-20488: SparkSubmitSparkClient#launchDriver should parse exceptions, not just errors (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master e8b87bfb0 -> d67d52c8d HIVE-20488: SparkSubmitSparkClient#launchDriver should parse exceptions, not just errors (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d67d52c8 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d67d52c8 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d67d52c8 Branch: refs/heads/master Commit: d67d52c8d56d41f5d8ca5c605ee10373dafb5bd1 Parents: e8b87bf Author: Bharathkrishna Guruvayoor Murali Authored: Thu Oct 18 10:36:02 2018 -0500 Committer: Sahil Takiar Committed: Thu Oct 18 10:36:41 2018 -0500 -- .../apache/hive/spark/client/SparkClientUtilities.java | 7 ++- .../apache/hive/spark/client/SparkSubmitSparkClient.java | 5 ++--- .../org/apache/hive/spark/client/TestSparkClient.java| 11 +++ 3 files changed, 19 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d67d52c8/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java index d3cb3dd..b434d8f 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java @@ -19,6 +19,7 @@ package org.apache.hive.spark.client; import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.File; @@ -49,7 +50,7 @@ public class SparkClientUtilities { public static final String HIVE_KRYO_REG_NAME = "org.apache.hive.spark.HiveKryoRegistrator"; private static final String HIVE_KRYO_REG_JAR_NAME = "hive-kryo-registrator"; - + private static final ImmutableList ERROR_KEYWORDS = ImmutableList.of("error", "exception"); /** * Add new elements to the classpath. * @@ -186,4 +187,8 @@ public class SparkClientUtilities { Thread.currentThread().setContextClassLoader(newLoader); } } + + public static boolean containsErrorKeyword(String line) { +return ERROR_KEYWORDS.stream().anyMatch(x -> StringUtils.containsIgnoreCase(line, x)); + } } http://git-wip-us.apache.org/repos/asf/hive/blob/d67d52c8/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java index 1879829..f42fee2 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java @@ -34,8 +34,6 @@ import java.util.concurrent.FutureTask; import java.util.regex.Pattern; import java.util.regex.Matcher; -import org.apache.commons.lang3.StringUtils; - import org.apache.hadoop.hive.common.log.LogRedirector; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; @@ -44,6 +42,7 @@ import org.apache.hive.spark.client.rpc.RpcServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hive.spark.client.SparkClientUtilities.containsErrorKeyword; /** * Extends the {@link AbstractSparkClient} and launches a child process to run Spark's {@code @@ -228,7 +227,7 @@ class SparkSubmitSparkClient extends AbstractSparkClient { List errorMessages = new ArrayList<>(); synchronized (childErrorLog) { for (String line : childErrorLog) { - if (StringUtils.containsIgnoreCase(line, "Error")) { + if (containsErrorKeyword(line)) { errorMessages.add("\"" + line + "\""); } } http://git-wip-us.apache.org/repos/asf/hive/blob/d67d52c8/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java -- diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java index 996b24e..0e1557e 100644 --- a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java +++ b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java @@ -301,6 +301,17 @@ public class TestSparkClient { }); } + @Test + public void
hive git commit: HIVE-17684: HoS memory issues with MapJoinMemoryExhaustionHandler (Misha Dmitriev, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 4137c212c -> a036e52df HIVE-17684: HoS memory issues with MapJoinMemoryExhaustionHandler (Misha Dmitriev, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a036e52d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a036e52d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a036e52d Branch: refs/heads/master Commit: a036e52dff85215850dee365e4093b436d301f42 Parents: 4137c21 Author: Misha Dmitriev Authored: Tue Sep 25 18:05:03 2018 -0700 Committer: Sahil Takiar Committed: Tue Sep 25 18:05:16 2018 -0700 -- .../hadoop/hive/common/GcTimeMonitor.java | 261 +++ .../org/apache/hadoop/hive/conf/HiveConf.java | 12 +- data/conf/hive-site.xml | 10 + data/conf/spark/standalone/hive-site.xml| 7 +- pom.xml | 1 + .../ql/exec/DefaultMemoryExhaustionChecker.java | 45 .../hive/ql/exec/HashTableSinkOperator.java | 9 +- .../hive/ql/exec/MemoryExhaustionChecker.java | 28 ++ .../ql/exec/MemoryExhaustionCheckerFactory.java | 39 +++ .../ql/exec/SparkMemoryExhaustionChecker.java | 91 +++ 10 files changed, 494 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a036e52d/common/src/java/org/apache/hadoop/hive/common/GcTimeMonitor.java -- diff --git a/common/src/java/org/apache/hadoop/hive/common/GcTimeMonitor.java b/common/src/java/org/apache/hadoop/hive/common/GcTimeMonitor.java new file mode 100644 index 000..edba6f9 --- /dev/null +++ b/common/src/java/org/apache/hadoop/hive/common/GcTimeMonitor.java @@ -0,0 +1,261 @@ +/** + * 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.common; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * Based on org.apache.hadoop.util.GcTimeMonitor. However, this class detects + * GC pauses using the same method as JvmPauseMonitor (by comparing the actual + * and expected thread sleep time) rather than by reading information from + * GarbageCollectionMXBean. The latter may sometimes report time spent in + * concurrent GC operations rather than GC pauses. This may result in inaccurate + * results when trying to estimate the time that the JVM is "frozen" due to GC. + * + * This class monitors the percentage of time the JVM is paused in GC within + * the specified observation window, say 1 minute. The user can provide a + * hook which will be called whenever this percentage exceeds the specified + * threshold. + */ +public class GcTimeMonitor extends Thread { + + private final long maxGcTimePercentage; + private final long observationWindowNanos, sleepIntervalMs; + private final GcTimeAlertHandler alertHandler; + + // Ring buffers containing GC timings and timestamps when timings were taken + private final TsAndData[] gcDataBuf; + private int bufSize, startIdx, endIdx; + + private long startTimeNanos; + private final GcData curData = new GcData(); + private volatile boolean shouldRun = true; + + /** + * Create an instance of GCTimeMonitor. Once it's started, it will stay alive + * and monitor GC time percentage until shutdown() is called. If you don't + * put a limit on the number of GCTimeMonitor instances that you create, and + * alertHandler != null, you should necessarily call shutdown() once the given + * instance is not needed. Otherwise, you may create a memory leak, because + * each running GCTimeMonitor will keep its alertHandler object in memory, + * which in turn may reference and keep in memory many more other objects. + * + * @param observationWindowMs the interval over which the percentage + * of GC time should be calculated. A practical value would be somewhere + * between 30 sec and several minutes. + * @param sleepIntervalMs how frequently this thread should
hive git commit: HIVE-19814: RPC Server port is always random for spark (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 84e5b9391 -> a3b7a2452 HIVE-19814: RPC Server port is always random for spark (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a3b7a245 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a3b7a245 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a3b7a245 Branch: refs/heads/master Commit: a3b7a2452bacf6d7eeeb42bd9dd68109c90e27a2 Parents: 84e5b93 Author: Bharathkrishna Guruvayoor Murali Authored: Wed Sep 12 17:50:29 2018 -0700 Committer: Sahil Takiar Committed: Wed Sep 12 17:50:54 2018 -0700 -- .../ql/exec/spark/HiveSparkClientFactory.java | 8 - .../session/TestSparkSessionManagerImpl.java| 35 .../hive/spark/client/SparkClientFactory.java | 6 .../hive/spark/client/rpc/RpcConfiguration.java | 17 +- 4 files changed, 57 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a3b7a245/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index 0aae0d8..a49e72d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.hive.common.LogUtils; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.spark.client.SparkClientUtilities; +import org.apache.hive.spark.client.rpc.RpcConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -198,7 +199,12 @@ public class HiveSparkClientFactory { LOG.debug(String.format( "Pass Oozie configuration (%s -> %s).", propertyName, LogUtils.maskIfPassword(propertyName,value))); } - + if (RpcConfiguration.HIVE_SPARK_RSC_CONFIGS.contains(propertyName)) { +String value = RpcConfiguration.getValue(hiveConf, propertyName); +sparkConf.put(propertyName, value); +LOG.debug(String.format("load RPC property from hive configuration (%s -> %s).", propertyName, +LogUtils.maskIfPassword(propertyName, value))); + } } final boolean optShuffleSerDe = hiveConf.getBoolVar( http://git-wip-us.apache.org/repos/asf/hive/blob/a3b7a245/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java index 853e4f4..3882b58 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java @@ -22,6 +22,8 @@ import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.util.StringUtils; +import org.apache.hive.spark.client.SparkClientFactory; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -31,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.TimeoutException; @@ -214,6 +217,38 @@ public class TestSparkSessionManagerImpl { assertEquals("0", ss.getSparkSession().getSessionId()); } + @Test + public void testConfigsForInitialization() { +//Test to make sure that configs listed in RpcConfiguration.HIVE_SPARK_RSC_CONFIGS which are passed +// through HiveConf are included in the Spark configuration. +HiveConf hiveConf = getHiveConf(); +hiveConf.setVar(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT, "49152-49222,49223,49224-49333"); +hiveConf.setVar(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS, "test-rpc-server-address"); +Map sparkConf = HiveSparkClientFactory.initiateSparkConf(hiveConf, null); +assertEquals("49152-49222,49223,49224-49333", sparkConf.get(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname)); +assertEquals("test-rpc-server-address", sparkConf.get(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname)); + } + + @Test + public void testServerPortAssignment() throws Exception { +HiveConf conf = getHiveConf(); +
hive git commit: HIVE-14162: Allow disabling of long running job on Hive On Spark On YARN (Sahil Takiar, reviewed by Adam Szita)
Repository: hive Updated Branches: refs/heads/master 219538701 -> 189d3fec2 HIVE-14162: Allow disabling of long running job on Hive On Spark On YARN (Sahil Takiar, reviewed by Adam Szita) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/189d3fec Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/189d3fec Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/189d3fec Branch: refs/heads/master Commit: 189d3fec25dfb94b209b1a34c1be674ce9d85bc5 Parents: 2195387 Author: Sahil Takiar Authored: Mon Jul 16 10:26:21 2018 -0500 Committer: Sahil Takiar Committed: Fri Aug 31 15:42:57 2018 -0500 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 6 + .../ql/exec/spark/TestSparkSessionTimeout.java | 145 +++ .../java/org/apache/hadoop/hive/ql/Driver.java | 10 + .../ql/exec/spark/session/SparkSession.java | 27 ++ .../ql/exec/spark/session/SparkSessionImpl.java | 246 ++- .../spark/session/SparkSessionManagerImpl.java | 63 +++-- 6 files changed, 423 insertions(+), 74 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/189d3fec/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 8c39de3..40ea3ac 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -4311,6 +4311,12 @@ public class HiveConf extends Configuration { "specified (default) then the spark-submit shell script is used to launch the Spark " + "app. If " + HIVE_SPARK_LAUNCHER_CLIENT + " is specified then Spark's " + "InProcessLauncher is used to programmatically launch the app."), +SPARK_SESSION_TIMEOUT("hive.spark.session.timeout", "30m", new TimeValidator(TimeUnit.MINUTES, +30L, true, null, true), "Amount of time the Spark Remote Driver should wait for " + +" a Spark job to be submitted before shutting down. Minimum value is 30 minutes"), +SPARK_SESSION_TIMEOUT_PERIOD("hive.spark.session.timeout.period", "60s", +new TimeValidator(TimeUnit.SECONDS, 60L, true, null, true), +"How frequently to check for idle Spark sessions. Minimum value is 60 seconds."), 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, http://git-wip-us.apache.org/repos/asf/hive/blob/189d3fec/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java new file mode 100644 index 000..c887297 --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.exec.spark; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; +import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.net.MalformedURLException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService;
hive git commit: HIVE-19008: Improve Spark session id logging (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 7cf791472 -> 7795c0a7d HIVE-19008: Improve Spark session id logging (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7795c0a7 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7795c0a7 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7795c0a7 Branch: refs/heads/master Commit: 7795c0a7dc59941671f8845d78b16d9e5ddc9ea3 Parents: 7cf7914 Author: Sahil Takiar Authored: Sun Apr 1 21:21:43 2018 -0500 Committer: Sahil Takiar Committed: Mon Aug 6 11:34:00 2018 +0200 -- .../ql/exec/spark/HiveSparkClientFactory.java | 13 +++--- .../ql/exec/spark/session/SparkSessionImpl.java | 11 ++--- .../spark/session/SparkSessionManagerImpl.java | 3 +- .../hadoop/hive/ql/session/SessionState.java| 6 +++ .../session/TestSparkSessionManagerImpl.java| 43 ++-- 5 files changed, 58 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7795c0a7/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index 5ed5d42..0aae0d8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -62,8 +62,9 @@ public class HiveSparkClientFactory { @VisibleForTesting public static final String SPARK_CLONE_CONFIGURATION = "spark.hadoop.cloneConf"; - public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf, String sessionId) throws Exception { -Map sparkConf = initiateSparkConf(hiveconf, sessionId); + public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf, String sparkSessionId, + String hiveSessionId) throws Exception { +Map sparkConf = initiateSparkConf(hiveconf, hiveSessionId); // Submit spark job through local spark context while spark master is local mode, otherwise submit // spark job through remote spark context. @@ -72,11 +73,11 @@ public class HiveSparkClientFactory { // With local spark context, all user sessions share the same spark context. return LocalHiveSparkClient.getInstance(generateSparkConf(sparkConf), hiveconf); } else { - return new RemoteHiveSparkClient(hiveconf, sparkConf, sessionId); + return new RemoteHiveSparkClient(hiveconf, sparkConf, hiveSessionId + "_" + sparkSessionId); } } - public static Map initiateSparkConf(HiveConf hiveConf, String sessionId) { + public static Map initiateSparkConf(HiveConf hiveConf, String hiveSessionId) { Map sparkConf = new HashMap(); HBaseConfiguration.addHbaseResources(hiveConf); @@ -84,9 +85,9 @@ public class HiveSparkClientFactory { sparkConf.put("spark.master", SPARK_DEFAULT_MASTER); final String appNameKey = "spark.app.name"; String appName = hiveConf.get(appNameKey); -final String sessionIdString = " (sessionId = " + sessionId + ")"; +final String sessionIdString = " (hiveSessionId = " + hiveSessionId + ")"; if (appName == null) { - if (sessionId == null) { + if (hiveSessionId == null) { appName = SPARK_DEFAULT_APP_NAME; } else { appName = SPARK_DEFAULT_APP_NAME + sessionIdString; http://git-wip-us.apache.org/repos/asf/hive/blob/7795c0a7/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java index 0f2f031..2015810 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java @@ -72,8 +72,8 @@ public class SparkSessionImpl implements SparkSession { private Path scratchDir; private final Object dirLock = new Object(); - public SparkSessionImpl() { -sessionId = makeSessionId(); + SparkSessionImpl(String sessionId) { +this.sessionId = sessionId; initErrorPatterns(); } @@ -83,7 +83,8 @@ public class SparkSessionImpl implements SparkSession { this.conf = conf; isOpen = true; try { - hiveSparkClient = HiveSparkClientFactory.createHiveSparkClient(conf, sessionId); + hiveSparkClient = HiveSparkClientFactory.createHiveSparkClient(conf, sessionId, +
hive git commit: HIVE-19937: Intern fields in MapWork on deserialization (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master 20c95c1c0 -> 7cf791472 HIVE-19937: Intern fields in MapWork on deserialization (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7cf79147 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7cf79147 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7cf79147 Branch: refs/heads/master Commit: 7cf7914729ceeca34017ad6671a97a1290915e10 Parents: 20c95c1 Author: Sahil Takiar Authored: Mon Jun 18 15:38:13 2018 -0500 Committer: Sahil Takiar Committed: Mon Aug 6 11:05:33 2018 +0200 -- .../hive/ql/exec/AbstractMapOperator.java | 4 +- .../apache/hadoop/hive/ql/exec/MapOperator.java | 13 ++--- .../hive/ql/exec/SerializationUtilities.java| 52 +++- .../hive/ql/exec/vector/VectorMapOperator.java | 12 ++--- .../hive/ql/io/parquet/ProjectionPusher.java| 4 +- .../org/apache/hadoop/hive/ql/plan/MapWork.java | 17 +-- .../hadoop/hive/ql/plan/PartitionDesc.java | 2 +- 7 files changed, 82 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7cf79147/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapOperator.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapOperator.java index 0d1c688..c7af991 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapOperator.java @@ -96,7 +96,7 @@ public abstract class AbstractMapOperator extends Operator return path; } - protected String getNominalPath(Path fpath) { + protected Path getNominalPath(Path fpath) { Path nominal = null; boolean schemaless = fpath.toUri().getScheme() == null; for (Path onefile : conf.getPathToAliases().keySet()) { @@ -119,7 +119,7 @@ public abstract class AbstractMapOperator extends Operator if (nominal == null) { throw new IllegalStateException("Invalid input path " + fpath); } -return nominal.toString(); +return nominal; } public abstract void initEmptyInputChildren(List> children, Configuration hconf) http://git-wip-us.apache.org/repos/asf/hive/blob/7cf79147/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java index c7350ca..b9986d3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java @@ -79,8 +79,7 @@ public class MapOperator extends AbstractMapOperator { protected transient long logEveryNRows = 0; // input path --> {operator --> context} - private final Map, MapOpCtx>> opCtxMap = - new HashMap, MapOpCtx>>(); + private final Map, MapOpCtx>> opCtxMap = new HashMap<>(); // child operator --> object inspector (converted OI if it's needed) private final Map, StructObjectInspector> childrenOpToOI = new HashMap, StructObjectInspector>(); @@ -440,10 +439,8 @@ public class MapOperator extends AbstractMapOperator { LOG.debug("Adding alias " + alias + " to work list for file " + onefile); } -Map, MapOpCtx> contexts = opCtxMap.get(onefile.toString()); -if (contexts == null) { - opCtxMap.put(onefile.toString(), contexts = new LinkedHashMap, MapOpCtx>()); -} +Map, MapOpCtx> contexts = opCtxMap.computeIfAbsent(onefile, +k -> new LinkedHashMap<>()); if (contexts.containsKey(op)) { continue; } @@ -515,7 +512,7 @@ public class MapOperator extends AbstractMapOperator { public void cleanUpInputFileChangedOp() throws HiveException { super.cleanUpInputFileChangedOp(); Path fpath = getExecContext().getCurrentInputPath(); -String nominalPath = getNominalPath(fpath); +Path nominalPath = getNominalPath(fpath); Map, MapOpCtx> contexts = opCtxMap.get(nominalPath); if (LOG.isInfoEnabled()) { StringBuilder builder = new StringBuilder(); @@ -703,7 +700,7 @@ public class MapOperator extends AbstractMapOperator { public void initializeContexts() { Path fpath = getExecContext().getCurrentInputPath(); -String nominalPath = getNominalPath(fpath); +Path nominalPath = getNominalPath(fpath); Map, MapOpCtx> contexts = opCtxMap.get(nominalPath); currentCtxs = contexts.values().toArray(new MapOpCtx[contexts.size()]); }
hive git commit: HIVE-20032: Don't serialize hashCode for repartitionAndSortWithinPartitions (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master 94ec36865 -> 1e437e2b1 HIVE-20032: Don't serialize hashCode for repartitionAndSortWithinPartitions (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1e437e2b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1e437e2b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1e437e2b Branch: refs/heads/master Commit: 1e437e2b1e00aa1942781f0a9b5b2f0868644b81 Parents: 94ec368 Author: Sahil Takiar Authored: Thu Jun 28 17:50:41 2018 -0700 Committer: Sahil Takiar Committed: Fri Jul 27 08:38:28 2018 -0500 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 2 +- .../hive/ql/exec/spark/TestSparkStatistics.java | 12 ++-- .../jdbc/TestJdbcWithLocalClusterSpark.java | 10 +-- .../jdbc/TestJdbcWithMiniHS2ErasureCoding.java | 11 ++-- ...stMultiSessionsHS2WithLocalClusterSpark.java | 10 +-- .../apache/hive/spark/HiveKryoRegistrator.java | 2 +- .../hive/spark/NoHashCodeKryoSerializer.java| 65 .../ql/exec/spark/ShuffleKryoSerializer.java| 62 +++ .../hive/ql/exec/spark/SortByShuffler.java | 12 +++- .../hive/ql/exec/spark/SparkPlanGenerator.java | 20 -- .../org/apache/hadoop/hive/ql/io/HiveKey.java | 4 ++ .../hive/ql/exec/spark/TestHiveSparkClient.java | 17 ++--- .../hive/ql/exec/spark/TestSparkPlan.java | 14 +++-- .../hive/spark/client/TestSparkClient.java | 9 ++- 14 files changed, 208 insertions(+), 42 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1e437e2b/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 9f1da60..39c77b3 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -4239,7 +4239,7 @@ public class HiveConf extends Configuration { "If this is set to true, mapjoin optimization in Hive/Spark will use statistics from\n" + "TableScan operators at the root of operator tree, instead of parent ReduceSink\n" + "operators of the Join operator."), -SPARK_OPTIMIZE_SHUFFLE_SERDE("hive.spark.optimize.shuffle.serde", false, +SPARK_OPTIMIZE_SHUFFLE_SERDE("hive.spark.optimize.shuffle.serde", true, "If this is set to true, Hive on Spark will register custom serializers for data types\n" + "in shuffle. This should result in less shuffled data."), SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout", http://git-wip-us.apache.org/repos/asf/hive/blob/1e437e2b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java index d383873..191d5f5 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java @@ -33,6 +33,9 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Assert; import org.junit.Test; +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -41,13 +44,10 @@ import java.util.stream.Collectors; public class TestSparkStatistics { @Test - public void testSparkStatistics() { + public void testSparkStatistics() throws MalformedURLException { +String confDir = "../../data/conf/spark/standalone/hive-site.xml"; +HiveConf.setHiveSiteLocation(new File(confDir).toURI().toURL()); HiveConf conf = new HiveConf(); -conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, -SQLStdHiveAuthorizerFactory.class.getName()); -conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); -conf.setVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "spark"); -conf.set("spark.master", "local-cluster[1,2,1024]"); conf.set("spark.local.dir", Paths.get(System.getProperty("test.tmp.dir"), "TestSparkStatistics-local-dir").toString()); http://git-wip-us.apache.org/repos/asf/hive/blob/1e437e2b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java -- diff --git
hive git commit: HIVE-19986: Add logging of runtime statistics indicating when Hdfs Erasure Coding is used by MR (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 68bdf9eb4 -> 14bb84088 HIVE-19986: Add logging of runtime statistics indicating when Hdfs Erasure Coding is used by MR (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/14bb8408 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/14bb8408 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/14bb8408 Branch: refs/heads/master Commit: 14bb84088f65467d2ea0cc828a48cd33e3ec Parents: 68bdf9e Author: Andrew Sherman Authored: Wed Jul 25 16:23:51 2018 -0500 Committer: Sahil Takiar Committed: Wed Jul 25 16:44:58 2018 -0500 -- .../jdbc/TestJdbcWithMiniHS2ErasureCoding.java | 55 .../org/apache/hadoop/hive/ql/MapRedStats.java | 43 ++- .../hive/ql/exec/mr/HadoopJobExecHelper.java| 2 +- .../hive/ql/processors/ErasureProcessor.java| 10 +++- .../apache/hadoop/hive/shims/Hadoop23Shims.java | 17 ++ .../apache/hadoop/hive/shims/HadoopShims.java | 10 6 files changed, 134 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/14bb8408/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java index b0a0145..efb3759 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java @@ -19,6 +19,8 @@ package org.apache.hive.jdbc; import java.io.IOException; +import java.io.StringWriter; +import java.io.Writer; import java.nio.file.Paths; import java.sql.Connection; import java.sql.DriverManager; @@ -31,11 +33,17 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.processors.ErasureProcessor; import org.apache.hadoop.hive.shims.HadoopShims; import org.apache.hadoop.hive.shims.HadoopShims.HdfsErasureCodingShim; import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hive.jdbc.miniHS2.MiniHS2; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.LoggerContext; +import org.apache.logging.log4j.core.appender.WriterAppender; +import org.apache.logging.log4j.core.config.Configuration; +import org.apache.logging.log4j.core.layout.PatternLayout; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -174,6 +182,53 @@ public class TestJdbcWithMiniHS2ErasureCoding { } /** + * Test MR stats. + */ + @Test + public void testMapRedStats() throws Exception { +// Do log4j magic to save log output +StringWriter writer = new StringWriter(); +Appender appender = addAppender(writer, "testMapRedStats"); +try (Statement stmt = hs2Conn.createStatement()) { + String table = "mapredstats"; + stmt.execute("set hive.execution.engine=mr"); + stmt.execute(" CREATE TABLE " + table + " (a int) STORED AS PARQUET"); + stmt.execute("INSERT INTO TABLE " + table + " VALUES (3)"); + try (ResultSet rs = stmt.executeQuery("select a from " + table + " order by a")) { +while (rs.next()) { + int val = rs.getInt(1); + assertEquals(3, val); +} + } +} +String output = writer.toString(); +// check for standard stats +assertTrue(output.contains("HDFS Read:")); +assertTrue(output.contains("HDFS Write:")); + +// check for erasure coding stat +HadoopShims.HdfsErasureCodingShim erasureShim = ErasureProcessor.getErasureShim(conf); +if (erasureShim.isMapReduceStatAvailable()) { + assertTrue(output.contains("HDFS EC Read:")); +} + } + + /** + * Add an appender to log4j. + * http://logging.apache.org/log4j/2.x/manual/customconfig.html#AddingToCurrent + */ + private Appender addAppender(final Writer writer, final String writerName) { +final LoggerContext context = LoggerContext.getContext(false); +final Configuration config = context.getConfiguration(); +final PatternLayout layout = PatternLayout.createDefaultLayout(config); +final Appender appender = +WriterAppender.createAppender(layout, null, writer, writerName, false, true); +appender.start(); +config.getRootLogger().addAppender(appender, null, null); +return appender; + } + + /** * Add a Erasure Coding Policy
hive git commit: HIVE-19766: Show the number of rows inserted when execution engine is Spark (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 758ff4490 -> a15d75b47 HIVE-19766: Show the number of rows inserted when execution engine is Spark (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a15d75b4 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a15d75b4 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a15d75b4 Branch: refs/heads/master Commit: a15d75b47af7acbe567d30c865e05f21c5ca7229 Parents: 758ff44 Author: Bharathkrishna Guruvayoor Murali Authored: Wed Jul 25 14:11:47 2018 -0500 Committer: Sahil Takiar Committed: Wed Jul 25 14:11:47 2018 -0500 -- .../apache/hadoop/hive/ql/exec/spark/SparkTask.java | 14 ++ 1 file changed, 14 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/a15d75b4/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index ad5049a..9277510 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -38,6 +38,8 @@ import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; import org.apache.hadoop.hive.ql.exec.spark.status.impl.SparkMetricsUtils; import org.apache.hadoop.hive.ql.exec.spark.status.SparkStage; +import org.apache.hive.spark.counter.SparkCounter; +import org.apache.hive.spark.counter.SparkCounters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -163,6 +165,17 @@ public class SparkTask extends Task { if (rc == 0) { sparkStatistics = sparkJobStatus.getSparkStatistics(); +if (SessionState.get() != null) { + //Set the number of rows written in case of insert queries, to print in the client(beeline). + SparkCounters counters = sparkJobStatus.getCounter(); + if (counters != null) { +SparkCounter counter = counters.getCounter(HiveConf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP), +FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN); +if (counter != null) { + queryState.setNumModifiedRows(counter.getValue()); +} + } +} printConsoleMetrics(); printExcessiveGCWarning(); if (LOG.isInfoEnabled() && sparkStatistics != null) { @@ -500,6 +513,7 @@ public class SparkTask extends Task { List hiveCounters = new LinkedList(); counters.put(groupName, hiveCounters); hiveCounters.add(Operator.HIVE_COUNTER_CREATED_FILES); +hiveCounters.add(FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN); // MapOperator is out of SparkWork, SparkMapRecordHandler use it to bridge // Spark transformation and Hive operators in SparkWork. for (MapOperator.Counter counter : MapOperator.Counter.values()) {
hive git commit: HIVE-19733: RemoteSparkJobStatus#getSparkStageProgress inefficient implementation (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 4e9562f1e -> ed4fa73ba HIVE-19733: RemoteSparkJobStatus#getSparkStageProgress inefficient implementation (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ed4fa73b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ed4fa73b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ed4fa73b Branch: refs/heads/master Commit: ed4fa73ba740026ac0d4297d6a45432dc60d1073 Parents: 4e9562f Author: Bharathkrishna Guruvayoor Murali Authored: Mon Jul 23 18:35:04 2018 -0500 Committer: Sahil Takiar Committed: Mon Jul 23 18:35:41 2018 -0500 -- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 3 + .../spark/status/impl/RemoteSparkJobStatus.java | 108 +-- 2 files changed, 78 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ed4fa73b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 06d0ed3..37bc153 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -599,6 +599,9 @@ public enum ErrorMsg { SPARK_GET_JOB_INFO_EXECUTIONERROR(30046, "Spark job failed in execution while getting job info due to exception {0}"), REPL_FILE_SYSTEM_OPERATION_RETRY(30047, "Replication file system operation retry expired."), + SPARK_GET_STAGES_INFO_TIMEOUT(30048, "Spark job GetSparkStagesInfoJob timed out after {0} seconds.", true), + SPARK_GET_STAGES_INFO_INTERRUPTED(30049, "Spark job GetSparkStagesInfoJob was interrupted."), + SPARK_GET_STAGES_INFO_EXECUTIONERROR(30050, "Spark job GetSparkStagesInfoJob failed in execution while getting job info due to exception {0}", true), //== 4 range starts here // http://git-wip-us.apache.org/repos/asf/hive/blob/ed4fa73b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java index 832832b..3d41443 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java @@ -45,6 +45,7 @@ import org.apache.spark.SparkStageInfo; import org.apache.spark.api.java.JavaFutureAction; import java.io.Serializable; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -103,18 +104,20 @@ public class RemoteSparkJobStatus implements SparkJobStatus { @Override public Map getSparkStageProgress() throws HiveException { +List sparkStagesInfo = getSparkStagesInfo(); Map stageProgresses = new HashMap(); -for (int stageId : getStageIds()) { - SparkStageInfo sparkStageInfo = getSparkStageInfo(stageId); - if (sparkStageInfo != null && sparkStageInfo.name() != null) { -int runningTaskCount = sparkStageInfo.numActiveTasks(); -int completedTaskCount = sparkStageInfo.numCompletedTasks(); -int failedTaskCount = sparkStageInfo.numFailedTasks(); -int totalTaskCount = sparkStageInfo.numTasks(); -SparkStageProgress sparkStageProgress = new SparkStageProgress( -totalTaskCount, completedTaskCount, runningTaskCount, failedTaskCount); -SparkStage stage = new SparkStage(sparkStageInfo.stageId(), sparkStageInfo.currentAttemptId()); -stageProgresses.put(stage, sparkStageProgress); +if (sparkStagesInfo != null) { + for (SparkStageInfo sparkStageInfo : sparkStagesInfo) { +if (sparkStageInfo != null && sparkStageInfo.name() != null) { + int runningTaskCount = sparkStageInfo.numActiveTasks(); + int completedTaskCount = sparkStageInfo.numCompletedTasks(); + int failedTaskCount = sparkStageInfo.numFailedTasks(); + int totalTaskCount = sparkStageInfo.numTasks(); + SparkStageProgress sparkStageProgress = + new SparkStageProgress(totalTaskCount, completedTaskCount, runningTaskCount, failedTaskCount); + SparkStage stage = new SparkStage(sparkStageInfo.stageId(), sparkStageInfo.currentAttemptId()); + stageProgresses.put(stage, sparkStageProgress); +} } } return stageProgresses; @@ -212,14 +215,26 @@ public class RemoteSparkJobStatus implements
hive git commit: HIVE-20056: SparkPartitionPruner shouldn't be triggered by Spark tasks (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master 6b1581623 -> bed17e54d HIVE-20056: SparkPartitionPruner shouldn't be triggered by Spark tasks (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bed17e54 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bed17e54 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bed17e54 Branch: refs/heads/master Commit: bed17e54d8ab465256e41f2b796d69d21a075ea0 Parents: 6b15816 Author: Sahil Takiar Authored: Sat Jul 14 12:51:01 2018 -0500 Committer: Sahil Takiar Committed: Mon Jul 23 09:12:55 2018 -0500 -- .../apache/hadoop/hive/ql/log/PerfLogger.java | 2 ++ .../exec/spark/SparkDynamicPartitionPruner.java | 25 .../hive/ql/exec/spark/SparkPlanGenerator.java | 24 +++ .../hadoop/hive/ql/io/HiveInputFormat.java | 12 -- 4 files changed, 41 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/bed17e54/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java -- diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java index 111e614..0ee41c0 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java +++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java @@ -85,6 +85,8 @@ public class PerfLogger { public static final String SPARK_OPTIMIZE_OPERATOR_TREE = "SparkOptimizeOperatorTree"; public static final String SPARK_OPTIMIZE_TASK_TREE = "SparkOptimizeTaskTree"; public static final String SPARK_FLUSH_HASHTABLE = "SparkFlushHashTable."; + public static final String SPARK_DYNAMICALLY_PRUNE_PARTITIONS = + "SparkDynamicallyPrunePartitions."; public static final String FILE_MOVES = "FileMoves"; public static final String LOAD_TABLE = "LoadTable"; http://git-wip-us.apache.org/repos/asf/hive/blob/bed17e54/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java index 240fa09..b9285ac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java @@ -30,9 +30,10 @@ import java.util.List; import java.util.Map; import java.util.Set; -import com.clearspring.analytics.util.Preconditions; -import javolution.testing.AssertionException; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; +import org.apache.hadoop.hive.ql.session.SessionState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; @@ -63,7 +64,11 @@ import org.apache.hadoop.util.ReflectionUtils; * The spark version of DynamicPartitionPruner. */ public class SparkDynamicPartitionPruner { + private static final Logger LOG = LoggerFactory.getLogger(SparkDynamicPartitionPruner.class); + private static final String CLASS_NAME = SparkDynamicPartitionPruner.class.getName(); + + private final PerfLogger perfLogger = SessionState.getPerfLogger(); private final Map> sourceInfoMap = new LinkedHashMap>(); private final BytesWritable writable = new BytesWritable(); @@ -74,8 +79,12 @@ public class SparkDynamicPartitionPruner { // Nothing to prune for this MapWork return; } +perfLogger.PerfLogBegin(CLASS_NAME, +PerfLogger.SPARK_DYNAMICALLY_PRUNE_PARTITIONS + work.getName()); processFiles(work, jobConf); prunePartitions(work); +perfLogger.PerfLogBegin(CLASS_NAME, +PerfLogger.SPARK_DYNAMICALLY_PRUNE_PARTITIONS + work.getName()); } public void initialize(MapWork work, JobConf jobConf) throws SerDeException { @@ -210,14 +219,11 @@ public class SparkDynamicPartitionPruner { Path p = it.next(); PartitionDesc desc = work.getPathToPartitionInfo().get(p); Map spec = desc.getPartSpec(); - if (spec == null) { -throw new AssertionException("No partition spec found in dynamic pruning"); - } + Preconditions.checkNotNull(spec, "No partition spec found in dynamic pruning"); String partValueString = spec.get(columnName); - if (partValueString == null) { -throw new AssertionException("Could not find partition value for column: " + columnName); - } +
[2/2] hive git commit: HIVE-19765: Add Parquet specific tests to BlobstoreCliDriver (Sahil Takiar, reviewed by Marta Kuczora)
HIVE-19765: Add Parquet specific tests to BlobstoreCliDriver (Sahil Takiar, reviewed by Marta Kuczora) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b574bcdd Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b574bcdd Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b574bcdd Branch: refs/heads/master Commit: b574bcdd01c154c46c7ae95b9b5d3dd57f5e32f5 Parents: c4c55d2 Author: Sahil Takiar Authored: Fri Jul 6 10:25:37 2018 -0500 Committer: Sahil Takiar Committed: Tue Jul 10 18:46:08 2018 -0500 -- data/files/part.parquet | Bin 0 -> 4758 bytes .../queries/clientpositive/parquet_buckets.q| 31 ++ .../clientpositive/parquet_format_nonpart.q | 30 ++ .../clientpositive/parquet_format_part.q| 67 +++ .../parquet_nonstd_partitions_loc.q | 100 .../queries/clientpositive/ptf_parquetfile.q| 32 ++ .../clientpositive/parquet_buckets.q.out| 183 +++ .../clientpositive/parquet_format_nonpart.q.out | 201 +++ .../clientpositive/parquet_format_part.q.out| 290 ++ .../parquet_nonstd_partitions_loc.q.out | 537 +++ .../clientpositive/ptf_parquetfile.q.out| 104 11 files changed, 1575 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b574bcdd/data/files/part.parquet -- diff --git a/data/files/part.parquet b/data/files/part.parquet new file mode 100644 index 000..5b48321 Binary files /dev/null and b/data/files/part.parquet differ http://git-wip-us.apache.org/repos/asf/hive/blob/b574bcdd/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q -- diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q new file mode 100644 index 000..c5a2cd3 --- /dev/null +++ b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q @@ -0,0 +1,31 @@ +-- Test simple interaction with partitioned bucketed table with paquet format in blobstore + +SET hive.exec.dynamic.partition=true; +SET hive.exec.reducers.max=10; +SET hive.exec.dynamic.partition.mode=nonstrict; + +DROP TABLE blobstore_source; +CREATE TABLE blobstore_source(a STRING, b STRING, c DOUBLE) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY ' ' +COLLECTION ITEMS TERMINATED BY '\t' +LINES TERMINATED BY '\n' +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_buckets/blobstore_source/'; +LOAD DATA LOCAL INPATH '../../data/files/3col_data.txt' INTO TABLE blobstore_source; + +DROP TABLE parquet_buckets; +CREATE TABLE parquet_buckets (a STRING, value DOUBLE) +PARTITIONED BY (b STRING) +CLUSTERED BY (a) INTO 10 BUCKETS +STORED AS PARQUET +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_buckets/parquet_buckets'; + +INSERT OVERWRITE TABLE parquet_buckets +PARTITION (b) +SELECT a, c, b FROM blobstore_source; +SELECT * FROM parquet_buckets; + +INSERT INTO TABLE parquet_buckets +PARTITION (b) +SELECT a, c, b FROM blobstore_source; +SELECT * FROM parquet_buckets; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/b574bcdd/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q -- diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q new file mode 100644 index 000..a7827e9 --- /dev/null +++ b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q @@ -0,0 +1,30 @@ +-- Test INSERT OVERWRITE and INSERT INTO on parquet table in blobstore + +DROP TABLE blobstore_source; +CREATE TABLE blobstore_source(a STRING, b STRING, c DOUBLE) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY ' ' +COLLECTION ITEMS TERMINATED BY '\t' +LINES TERMINATED BY '\n' +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_format_nonpart/blobstore_source/'; +LOAD DATA LOCAL INPATH '../../data/files/3col_data.txt' INTO TABLE blobstore_source; + +DROP TABLE parquet_table; +CREATE EXTERNAL TABLE parquet_table (a INT, b STRING, value DOUBLE) STORED AS PARQUET +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_format_nonpart/parquet_table'; + +INSERT OVERWRITE TABLE parquet_table +SELECT * FROM blobstore_source; + +SELECT * FROM parquet_table; +SELECT a FROM parquet_table GROUP BY a; +SELECT b FROM parquet_table GROUP BY b; +SELECT value FROM parquet_table GROUP BY value; + +INSERT INTO TABLE parquet_table +SELECT * FROM blobstore_source; + +SELECT * FROM parquet_table; +SELECT a FROM parquet_table GROUP
[1/2] hive git commit: HIVE-19765: Add Parquet specific tests to BlobstoreCliDriver (Sahil Takiar, reviewed by Marta Kuczora)
Repository: hive Updated Branches: refs/heads/master c4c55d2ab -> b574bcdd0 http://git-wip-us.apache.org/repos/asf/hive/blob/b574bcdd/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out -- diff --git a/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out b/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out new file mode 100644 index 000..15ae3d9 --- /dev/null +++ b/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out @@ -0,0 +1,537 @@ +PREHOOK: query: DROP TABLE src_events +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE src_events +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE src_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT, + run_dateSTRING, + game_id INT, + event_name STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + A masked pattern was here +PREHOOK: type: CREATETABLE +PREHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/src_events +PREHOOK: Output: database:default +PREHOOK: Output: default@src_events +POSTHOOK: query: CREATE TABLE src_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT, + run_dateSTRING, + game_id INT, + event_name STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + A masked pattern was here +POSTHOOK: type: CREATETABLE +POSTHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/src_events +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_events +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/events.txt' INTO TABLE src_events +PREHOOK: type: LOAD + A masked pattern was here +PREHOOK: Output: default@src_events +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/events.txt' INTO TABLE src_events +POSTHOOK: type: LOAD + A masked pattern was here +POSTHOOK: Output: default@src_events +PREHOOK: query: DROP TABLE parquet_events +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE parquet_events +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE parquet_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT +) +PARTITIONED BY (run_date STRING, game_id INT, event_name STRING) +STORED AS PARQUET + A masked pattern was here +PREHOOK: type: CREATETABLE +PREHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/parquet_events +PREHOOK: Output: database:default +PREHOOK: Output: default@parquet_events +POSTHOOK: query: CREATE TABLE parquet_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT +) +PARTITIONED BY (run_date STRING, game_id INT, event_name STRING) +STORED AS PARQUET + A masked pattern was here +POSTHOOK: type: CREATETABLE +POSTHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/parquet_events +POSTHOOK: Output: database:default +POSTHOOK: Output: default@parquet_events +PREHOOK: query: INSERT OVERWRITE TABLE parquet_events PARTITION (run_date, game_id, event_name) +SELECT * FROM src_events +PREHOOK: type: QUERY +PREHOOK: Input: default@src_events +PREHOOK: Output: default@parquet_events +POSTHOOK: query: INSERT OVERWRITE TABLE parquet_events PARTITION (run_date, game_id, event_name) +SELECT * FROM src_events +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_events +POSTHOOK: Output: default@parquet_events@run_date=20120921/game_id=39/event_name=hq_change +POSTHOOK: Output: default@parquet_events@run_date=20121021/game_id=39/event_name=hq_change +POSTHOOK: Output: default@parquet_events@run_date=20121121/game_id=39/event_name=hq_change +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).event_data SIMPLE [(src_events)src_events.FieldSchema(name:event_data, type:string, comment:null), ] +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).full_uid SIMPLE [(src_events)src_events.FieldSchema(name:full_uid, type:bigint, comment:null), ] +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).log_id SIMPLE [(src_events)src_events.FieldSchema(name:log_id, type:bigint, comment:null), ] +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).session_id SIMPLE
hive git commit: HIVE-20077: hcat command should follow same pattern as hive cli for getting HBase jars (Sean Busbey, reviewed by Aihua Xu, Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 403c1ea9c -> 02a038e23 HIVE-20077: hcat command should follow same pattern as hive cli for getting HBase jars (Sean Busbey, reviewed by Aihua Xu, Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/02a038e2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/02a038e2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/02a038e2 Branch: refs/heads/master Commit: 02a038e230e38685d8050a09b1f578f259d578c8 Parents: 403c1ea Author: Sean Busbey Authored: Fri Jul 6 11:09:57 2018 -0500 Committer: Sahil Takiar Committed: Fri Jul 6 11:09:57 2018 -0500 -- hcatalog/bin/hcat | 41 ++--- 1 file changed, 30 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/02a038e2/hcatalog/bin/hcat -- diff --git a/hcatalog/bin/hcat b/hcatalog/bin/hcat index 7889263..2211407 100644 --- a/hcatalog/bin/hcat +++ b/hcatalog/bin/hcat @@ -145,18 +145,37 @@ fi # Put external jars, hcat jar, and config file in the classpath HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:${HCAT_CLASSPATH}:${HCAT_JAR}:${HIVE_CONF_DIR} -# Look for HBase in a BigTop-compatible way. Avoid thrift version -# conflict with modern versions of HBase. +# HBase detection. Need bin/hbase and a conf dir for building classpath entries. +# Start with BigTop defaults for HBASE_HOME and HBASE_CONF_DIR. HBASE_HOME=${HBASE_HOME:-"/usr/lib/hbase"} -HBASE_CONF_DIR=${HBASE_CONF_DIR:-"${HBASE_HOME}/conf"} -if [ -d ${HBASE_HOME} ] ; then - for jar in $(find $HBASE_HOME -name '*.jar' -not -name '*thrift*'); do - HBASE_CLASSPATH=$HBASE_CLASSPATH:${jar} - done - export HADOOP_CLASSPATH="${HADOOP_CLASSPATH}:${HBASE_CLASSPATH}" -fi -if [ -d $HBASE_CONF_DIR ] ; then -HADOOP_CLASSPATH="${HADOOP_CLASSPATH}:${HBASE_CONF_DIR}" +HBASE_CONF_DIR=${HBASE_CONF_DIR:-"/etc/hbase/conf"} +if [[ ! -d $HBASE_CONF_DIR ]] ; then + # not explicitly set, nor in BigTop location. Try looking in HBASE_HOME. + HBASE_CONF_DIR="$HBASE_HOME/conf" +fi + +# perhaps we've located the HBase config. if so, include it on classpath. +if [[ -d $HBASE_CONF_DIR ]] ; then + export HADOOP_CLASSPATH="${HADOOP_CLASSPATH}:${HBASE_CONF_DIR}" +fi + +# look for the hbase script. First check HBASE_HOME and then ask PATH. +if [[ -e $HBASE_HOME/bin/hbase ]] ; then + HBASE_BIN="$HBASE_HOME/bin/hbase" +fi +HBASE_BIN=${HBASE_BIN:-"$(which hbase)"} + +# perhaps we've located HBase. If so, include its details on the classpath +if [[ -n $HBASE_BIN ]] ; then + # exclude ZK, PB, and Guava (See HIVE-2055) + # depends on HBASE-8438 (hbase-0.94.14+, hbase-0.96.1+) for `hbase mapredcp` command + for x in $($HBASE_BIN mapredcp 2>&2 | tr ':' '\n') ; do +if [[ $x == *zookeeper* || $x == *protobuf-java* || $x == *guava* ]] ; then + continue +fi +# TODO: should these should be added to AUX_PARAM as well? +export HADOOP_CLASSPATH="${HADOOP_CLASSPATH}:${x}" + done fi export HADOOP_CLASSPATH=$HADOOP_CLASSPATH
[2/2] hive git commit: HIVE-19765: Add Parquet specific tests to BlobstoreCliDriver (Sahil Takiar, reviewed by Marta Kuczora)
HIVE-19765: Add Parquet specific tests to BlobstoreCliDriver (Sahil Takiar, reviewed by Marta Kuczora) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cbcfab9e Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cbcfab9e Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cbcfab9e Branch: refs/heads/master Commit: cbcfab9eb1c58e0c389b093af188d6483440beb0 Parents: c2940a0 Author: Sahil Takiar Authored: Fri Jun 1 12:25:01 2018 -0500 Committer: Sahil Takiar Committed: Thu Jul 5 17:47:42 2018 -0500 -- .../queries/clientpositive/parquet_buckets.q| 31 ++ .../clientpositive/parquet_format_nonpart.q | 30 ++ .../clientpositive/parquet_format_part.q| 67 +++ .../parquet_nonstd_partitions_loc.q | 100 .../queries/clientpositive/ptf_parquetfile.q| 32 ++ .../clientpositive/parquet_buckets.q.out| 183 +++ .../clientpositive/parquet_format_nonpart.q.out | 201 +++ .../clientpositive/parquet_format_part.q.out| 290 ++ .../parquet_nonstd_partitions_loc.q.out | 537 +++ .../clientpositive/ptf_parquetfile.q.out| 104 10 files changed, 1575 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/cbcfab9e/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q -- diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q new file mode 100644 index 000..c5a2cd3 --- /dev/null +++ b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_buckets.q @@ -0,0 +1,31 @@ +-- Test simple interaction with partitioned bucketed table with paquet format in blobstore + +SET hive.exec.dynamic.partition=true; +SET hive.exec.reducers.max=10; +SET hive.exec.dynamic.partition.mode=nonstrict; + +DROP TABLE blobstore_source; +CREATE TABLE blobstore_source(a STRING, b STRING, c DOUBLE) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY ' ' +COLLECTION ITEMS TERMINATED BY '\t' +LINES TERMINATED BY '\n' +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_buckets/blobstore_source/'; +LOAD DATA LOCAL INPATH '../../data/files/3col_data.txt' INTO TABLE blobstore_source; + +DROP TABLE parquet_buckets; +CREATE TABLE parquet_buckets (a STRING, value DOUBLE) +PARTITIONED BY (b STRING) +CLUSTERED BY (a) INTO 10 BUCKETS +STORED AS PARQUET +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_buckets/parquet_buckets'; + +INSERT OVERWRITE TABLE parquet_buckets +PARTITION (b) +SELECT a, c, b FROM blobstore_source; +SELECT * FROM parquet_buckets; + +INSERT INTO TABLE parquet_buckets +PARTITION (b) +SELECT a, c, b FROM blobstore_source; +SELECT * FROM parquet_buckets; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/cbcfab9e/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q -- diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q new file mode 100644 index 000..a7827e9 --- /dev/null +++ b/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_nonpart.q @@ -0,0 +1,30 @@ +-- Test INSERT OVERWRITE and INSERT INTO on parquet table in blobstore + +DROP TABLE blobstore_source; +CREATE TABLE blobstore_source(a STRING, b STRING, c DOUBLE) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY ' ' +COLLECTION ITEMS TERMINATED BY '\t' +LINES TERMINATED BY '\n' +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_format_nonpart/blobstore_source/'; +LOAD DATA LOCAL INPATH '../../data/files/3col_data.txt' INTO TABLE blobstore_source; + +DROP TABLE parquet_table; +CREATE EXTERNAL TABLE parquet_table (a INT, b STRING, value DOUBLE) STORED AS PARQUET +LOCATION '${hiveconf:test.blobstore.path.unique}/parquet_format_nonpart/parquet_table'; + +INSERT OVERWRITE TABLE parquet_table +SELECT * FROM blobstore_source; + +SELECT * FROM parquet_table; +SELECT a FROM parquet_table GROUP BY a; +SELECT b FROM parquet_table GROUP BY b; +SELECT value FROM parquet_table GROUP BY value; + +INSERT INTO TABLE parquet_table +SELECT * FROM blobstore_source; + +SELECT * FROM parquet_table; +SELECT a FROM parquet_table GROUP BY a; +SELECT b FROM parquet_table GROUP BY b; +SELECT value FROM parquet_table GROUP BY value; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/cbcfab9e/itests/hive-blobstore/src/test/queries/clientpositive/parquet_format_part.q -- diff --git
[1/2] hive git commit: HIVE-19765: Add Parquet specific tests to BlobstoreCliDriver (Sahil Takiar, reviewed by Marta Kuczora)
Repository: hive Updated Branches: refs/heads/master c2940a07c -> cbcfab9eb http://git-wip-us.apache.org/repos/asf/hive/blob/cbcfab9e/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out -- diff --git a/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out b/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out new file mode 100644 index 000..15ae3d9 --- /dev/null +++ b/itests/hive-blobstore/src/test/results/clientpositive/parquet_nonstd_partitions_loc.q.out @@ -0,0 +1,537 @@ +PREHOOK: query: DROP TABLE src_events +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE src_events +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE src_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT, + run_dateSTRING, + game_id INT, + event_name STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + A masked pattern was here +PREHOOK: type: CREATETABLE +PREHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/src_events +PREHOOK: Output: database:default +PREHOOK: Output: default@src_events +POSTHOOK: query: CREATE TABLE src_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT, + run_dateSTRING, + game_id INT, + event_name STRING +) +ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + A masked pattern was here +POSTHOOK: type: CREATETABLE +POSTHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/src_events +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_events +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/events.txt' INTO TABLE src_events +PREHOOK: type: LOAD + A masked pattern was here +PREHOOK: Output: default@src_events +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/events.txt' INTO TABLE src_events +POSTHOOK: type: LOAD + A masked pattern was here +POSTHOOK: Output: default@src_events +PREHOOK: query: DROP TABLE parquet_events +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE parquet_events +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE parquet_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT +) +PARTITIONED BY (run_date STRING, game_id INT, event_name STRING) +STORED AS PARQUET + A masked pattern was here +PREHOOK: type: CREATETABLE +PREHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/parquet_events +PREHOOK: Output: database:default +PREHOOK: Output: default@parquet_events +POSTHOOK: query: CREATE TABLE parquet_events +( + log_id BIGINT, + `time`BIGINT, + uid BIGINT, + user_id BIGINT, + typeINT, + event_data STRING, + session_id STRING, + full_uidBIGINT +) +PARTITIONED BY (run_date STRING, game_id INT, event_name STRING) +STORED AS PARQUET + A masked pattern was here +POSTHOOK: type: CREATETABLE +POSTHOOK: Input: ### test.blobstore.path ###/parquet_nonstd_partitions_loc/parquet_events +POSTHOOK: Output: database:default +POSTHOOK: Output: default@parquet_events +PREHOOK: query: INSERT OVERWRITE TABLE parquet_events PARTITION (run_date, game_id, event_name) +SELECT * FROM src_events +PREHOOK: type: QUERY +PREHOOK: Input: default@src_events +PREHOOK: Output: default@parquet_events +POSTHOOK: query: INSERT OVERWRITE TABLE parquet_events PARTITION (run_date, game_id, event_name) +SELECT * FROM src_events +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src_events +POSTHOOK: Output: default@parquet_events@run_date=20120921/game_id=39/event_name=hq_change +POSTHOOK: Output: default@parquet_events@run_date=20121021/game_id=39/event_name=hq_change +POSTHOOK: Output: default@parquet_events@run_date=20121121/game_id=39/event_name=hq_change +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).event_data SIMPLE [(src_events)src_events.FieldSchema(name:event_data, type:string, comment:null), ] +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).full_uid SIMPLE [(src_events)src_events.FieldSchema(name:full_uid, type:bigint, comment:null), ] +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).log_id SIMPLE [(src_events)src_events.FieldSchema(name:log_id, type:bigint, comment:null), ] +POSTHOOK: Lineage: parquet_events PARTITION(run_date=20120921,game_id=39,event_name=hq_change).session_id SIMPLE
[1/2] hive git commit: HIVE-18118: Explain Extended should indicate if a file being read is an EC file (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master e865b449e -> c2940a07c http://git-wip-us.apache.org/repos/asf/hive/blob/c2940a07/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java -- diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java index 55ff150..d5ae5d1 100644 --- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java +++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore.utils; import com.google.common.collect.ImmutableMap; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest; @@ -43,6 +44,7 @@ import java.util.Map; import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE; import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_FILES; +import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_ERASURE_CODED_FILES; import static org.apache.hadoop.hive.common.StatsSetupConst.STATS_GENERATED; import static org.apache.hadoop.hive.common.StatsSetupConst.TOTAL_SIZE; import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.updateTableStatsSlow; @@ -60,7 +62,11 @@ public class TestMetaStoreUtils { private static final String DB_NAME = "db1"; private static final String TABLE_NAME = "tbl1"; - private final Map paramsWithStats = ImmutableMap.of(NUM_FILES, "1", TOTAL_SIZE, "2"); + private final Map paramsWithStats = ImmutableMap.of( + NUM_FILES, "1", + TOTAL_SIZE, "2", + NUM_ERASURE_CODED_FILES, "0" + ); private Database db; @@ -120,7 +126,7 @@ public class TestMetaStoreUtils { * Create database * Create unpartitioned table * Create unpartitioned table which has params - * Call updateTableStatsSlow with arguments which should caue stats calculation + * Call updateTableStatsSlow with arguments which should cause stats calculation * Verify table statistics using mocked warehouse * Create table which already have stats * Call updateTableStatsSlow forcing stats recompute @@ -141,18 +147,17 @@ public class TestMetaStoreUtils { // Set up mock warehouse -FileStatus fs1 = new FileStatus(1, true, 2, 3, -4, new Path("/tmp/0")); -FileStatus fs2 = new FileStatus(fileLength, false, 3, 4, -5, new Path("/tmp/1")); -FileStatus fs3 = new FileStatus(fileLength, false, 3, 4, -5, new Path("/tmp/1")); +FileStatus fs1 = getFileStatus(1, true, 2, 3, 4, "/tmp/0", false); +FileStatus fs2 = getFileStatus(fileLength, false, 3, 4, 5, "/tmp/1", true); +FileStatus fs3 = getFileStatus(fileLength, false, 3, 4, 5, "/tmp/1", false); List fileStatus = Arrays.asList(fs1, fs2, fs3); Warehouse wh = mock(Warehouse.class); when(wh.getFileStatusesForUnpartitionedTable(db, tbl)).thenReturn(fileStatus); Map expected = ImmutableMap.of(NUM_FILES, "2", -TOTAL_SIZE, String.valueOf(2 * fileLength)); +TOTAL_SIZE, String.valueOf(2 * fileLength), +NUM_ERASURE_CODED_FILES, "1" +); updateTableStatsSlow(db, tbl, wh, false, false, null); assertThat(tbl.getParameters(), is(expected)); @@ -195,6 +200,7 @@ public class TestMetaStoreUtils { Map expected1 = ImmutableMap.of(NUM_FILES, "2", TOTAL_SIZE, String.valueOf(2 * fileLength), +NUM_ERASURE_CODED_FILES, "1", COLUMN_STATS_ACCURATE, "{\"BASIC_STATS\":\"true\"}"); assertThat(tbl3.getParameters(), is(expected1)); } @@ -227,7 +233,7 @@ public class TestMetaStoreUtils { } /** - * Verify that updateTableStatsSlow() does not calculate tabe statistics when + * Verify that updateTableStatsSlow() does not calculate table statistics when * * newDir is true * Table is partitioned @@ -270,5 +276,16 @@ public class TestMetaStoreUtils { updateTableStatsSlow(db, tbl2, wh, false, false, null); verify(wh, never()).getFileStatusesForUnpartitionedTable(db, tbl2); } + + /** + * Build a FileStatus object. + */ + private static FileStatus getFileStatus(long fileLength, boolean isdir, int blockReplication, + int blockSize, int modificationTime, String pathString, boolean isErasureCoded) { +return new FileStatus(fileLength, isdir, blockReplication, blockSize, modificationTime, +0L, (FsPermission)null, (String)null, (String)null, null, +new Path(pathString), false, false, isErasureCoded);
[2/2] hive git commit: HIVE-18118: Explain Extended should indicate if a file being read is an EC file (Andrew Sherman, reviewed by Sahil Takiar)
HIVE-18118: Explain Extended should indicate if a file being read is an EC file (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c2940a07 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c2940a07 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c2940a07 Branch: refs/heads/master Commit: c2940a07cf0891e922672782b73ec22551a7eedd Parents: e865b44 Author: Andrew Sherman Authored: Thu Jul 5 16:08:16 2018 -0500 Committer: Sahil Takiar Committed: Thu Jul 5 16:08:16 2018 -0500 -- .../hive/common/util/HiveStringUtils.java | 21 +- .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 2 +- .../jdbc/TestJdbcWithMiniHS2ErasureCoding.java | 215 ++ .../test/resources/testconfiguration.properties | 3 +- .../org/apache/hadoop/hive/ql/QTestUtil.java| 4 +- .../org/apache/hive/jdbc/miniHS2/MiniHS2.java | 23 +- .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 44 +- .../ql/metadata/SessionHiveMetaStoreClient.java | 2 +- .../formatting/MetaDataFormatUtils.java | 11 +- .../formatting/TextMetaDataFormatter.java | 10 + .../optimizer/spark/SparkMapJoinOptimizer.java | 2 +- .../stats/annotation/StatsRulesProcFactory.java | 2 +- .../hadoop/hive/ql/plan/PartitionDesc.java | 4 +- .../apache/hadoop/hive/ql/plan/PlanUtils.java | 31 ++ .../apache/hadoop/hive/ql/plan/Statistics.java | 12 +- .../apache/hadoop/hive/ql/plan/TableDesc.java | 3 +- .../hive/ql/stats/BasicStatsNoJobTask.java | 7 +- .../hadoop/hive/ql/stats/BasicStatsTask.java| 4 +- .../apache/hadoop/hive/ql/stats/StatsUtils.java | 20 +- .../hive/ql/txn/compactor/CompactorMR.java | 4 +- .../queries/clientpositive/erasure_explain.q| 24 ++ .../queries/clientpositive/erasure_simple.q | 13 + .../clientnegative/unset_table_property.q.out | 1 + .../materialized_view_create_rewrite.q.out | 2 + .../clientpositive/druid/druidmini_mv.q.out | 3 + .../erasurecoding/erasure_explain.q.out | 409 +++ .../erasurecoding/erasure_simple.q.out | 20 +- .../llap/materialized_view_create.q.out | 1 + .../llap/materialized_view_create_rewrite.q.out | 2 + ...materialized_view_create_rewrite_dummy.q.out | 2 + ...erialized_view_create_rewrite_multi_db.q.out | 2 + .../llap/materialized_view_describe.q.out | 1 + .../clientpositive/show_tblproperties.q.out | 5 + .../unset_table_view_property.q.out | 9 + .../hadoop/hive/common/StatsSetupConst.java | 21 +- .../hadoop/hive/metastore/HiveMetaStore.java| 3 +- .../hive/metastore/utils/MetaStoreUtils.java| 11 +- .../metastore/utils/TestMetaStoreUtils.java | 37 +- 38 files changed, 877 insertions(+), 113 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/c2940a07/common/src/java/org/apache/hive/common/util/HiveStringUtils.java -- diff --git a/common/src/java/org/apache/hive/common/util/HiveStringUtils.java b/common/src/java/org/apache/hive/common/util/HiveStringUtils.java index cfe9b22..6b14ad9 100644 --- a/common/src/java/org/apache/hive/common/util/HiveStringUtils.java +++ b/common/src/java/org/apache/hive/common/util/HiveStringUtils.java @@ -31,19 +31,15 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Date; +import java.util.HashMap; import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.HashMap; import java.util.Locale; -import java.util.Properties; +import java.util.Map; import java.util.StringTokenizer; import java.util.regex.Pattern; import com.google.common.base.Splitter; -import com.google.common.collect.Interner; -import com.google.common.collect.Interners; - import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.text.translate.CharSequenceTranslator; import org.apache.commons.lang3.text.translate.EntityArrays; @@ -1062,19 +1058,6 @@ public class HiveStringUtils { return identifier.trim().toLowerCase(); } - public static Map getPropertiesExplain(Properties properties) { -if (properties != null) { - String value = properties.getProperty("columns.comments"); - if (value != null) { -// should copy properties first -Map clone = new HashMap(properties); -clone.put("columns.comments", quoteComments(value)); -return clone; - } -} -return properties; - } - public static String quoteComments(String value) { char[] chars = value.toCharArray(); if (!commentProvided(chars)) {
hive git commit: HIVE-19176: Add HoS support to progress bar on Beeline client (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master e19b861cf -> e7d1781ec HIVE-19176: Add HoS support to progress bar on Beeline client (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e7d1781e Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e7d1781e Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e7d1781e Branch: refs/heads/master Commit: e7d1781ec4662e088dcd6ffbe3f866738792ad9b Parents: e19b861 Author: Bharathkrishna Guruvayoor Murali Authored: Mon Jul 2 11:42:59 2018 -0700 Committer: Sahil Takiar Committed: Mon Jul 2 11:42:59 2018 -0700 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 2 +- .../org/apache/hive/jdbc/HiveStatement.java | 4 +- .../exec/spark/status/LocalSparkJobMonitor.java | 4 +- .../spark/status/RemoteSparkJobMonitor.java | 4 +- .../ql/exec/spark/status/RenderStrategy.java| 246 +++ .../ql/exec/spark/status/SparkJobMonitor.java | 157 +--- .../hive/ql/exec/spark/TestSparkTask.java | 1 + .../exec/spark/status/TestSparkJobMonitor.java | 29 ++- .../org/apache/hive/service/ServiceUtils.java | 5 +- .../cli/SparkProgressMonitorStatusMapper.java | 52 .../service/cli/thrift/ThriftCLIService.java| 5 +- 11 files changed, 349 insertions(+), 160 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/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 a3dd53e..7ef22d6 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3780,7 +3780,7 @@ public class HiveConf extends Configuration { "hive.server2.in.place.progress", true, "Allows hive server 2 to send progress bar update information. This is currently available" -+ " only if the execution engine is tez."), ++ " only if the execution engine is tez or Spark."), TEZ_DAG_STATUS_CHECK_INTERVAL("hive.tez.dag.status.check.interval", "500ms", new TimeValidator(TimeUnit.MILLISECONDS), "Interval between subsequent DAG status invocation."), SPARK_EXEC_INPLACE_PROGRESS("hive.spark.exec.inplace.progress", true, http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java -- diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java index ad8d1a7..0b38f9c 100644 --- a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java +++ b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java @@ -376,7 +376,9 @@ public class HiveStatement implements java.sql.Statement { * essentially return after the HIVE_SERVER2_LONG_POLLING_TIMEOUT (a server config) expires */ statusResp = client.GetOperationStatus(statusReq); -inPlaceUpdateStream.update(statusResp.getProgressUpdateResponse()); +if(!isOperationComplete) { + inPlaceUpdateStream.update(statusResp.getProgressUpdateResponse()); +} Utils.verifySuccessWithInfo(statusResp.getStatus()); if (statusResp.isSetOperationState()) { switch (statusResp.getOperationState()) { http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java index 2a6c33b..aeef3c1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java @@ -89,11 +89,11 @@ public class LocalSparkJobMonitor extends SparkJobMonitor { + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); } -printStatus(progressMap, lastProgressMap); +updateFunction.printStatus(progressMap, lastProgressMap); lastProgressMap = progressMap; break; case SUCCEEDED: -printStatus(progressMap, lastProgressMap); +updateFunction.printStatus(progressMap, lastProgressMap); lastProgressMap = progressMap; double duration = (System.currentTimeMillis() -
hive git commit: HIVE-18916: SparkClientImpl doesn't error out if spark-submit fails (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 88da0e8b7 -> e19b861cf HIVE-18916: SparkClientImpl doesn't error out if spark-submit fails (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e19b861c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e19b861c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e19b861c Branch: refs/heads/master Commit: e19b861cfbcb15166f9255f8b375ff5d8056b417 Parents: 88da0e8 Author: Sahil Takiar Authored: Mon Jul 2 11:30:14 2018 -0700 Committer: Sahil Takiar Committed: Mon Jul 2 11:30:14 2018 -0700 -- .../test/resources/testconfiguration.properties | 4 +- .../apache/hadoop/hive/ql/QOutProcessor.java| 2 + .../hadoop/hive/ql/exec/spark/SparkTask.java| 15 ++-- .../ql/exec/spark/session/SparkSessionImpl.java | 10 ++- .../spark/status/RemoteSparkJobMonitor.java | 1 - .../exec/spark/TestSparkInvalidFileFormat.java | 81 .../spark_submit_negative_executor_cores.q | 5 ++ .../spark_submit_negative_executor_memory.q | 5 ++ .../spark_submit_negative_executor_cores.q.out | 5 ++ .../spark_submit_negative_executor_memory.q.out | 5 ++ .../hive/spark/client/AbstractSparkClient.java | 25 +++--- .../spark/client/SparkSubmitSparkClient.java| 22 +++--- .../apache/hive/spark/client/rpc/RpcServer.java | 21 +++-- 13 files changed, 159 insertions(+), 42 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index d415b7d..385b71e 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -1684,7 +1684,9 @@ spark.query.negative.files=groupby2_map_skew_multi_distinct.q,\ spark.only.query.negative.files=spark_job_max_tasks.q,\ spark_stage_max_tasks.q,\ - spark_task_failure.q + spark_task_failure.q,\ + spark_submit_negative_executor_cores.q,\ + spark_submit_negative_executor_memory.q spark.perf.disabled.query.files=query14.q,\ query64.q http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java -- diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java index 359f027..1e4cddd 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java @@ -273,6 +273,8 @@ public class QOutProcessor { ppm.add(new PatternReplacementPair(Pattern.compile("attempt_[0-9_]+"), "attempt_#ID#")); ppm.add(new PatternReplacementPair(Pattern.compile("vertex_[0-9_]+"), "vertex_#ID#")); ppm.add(new PatternReplacementPair(Pattern.compile("task_[0-9_]+"), "task_#ID#")); +ppm.add(new PatternReplacementPair(Pattern.compile("for Spark session.*?:"), +"#SPARK_SESSION_ID#:")); partialPlanMask = ppm.toArray(new PatternReplacementPair[ppm.size()]); } /* This list may be modified by specific cli drivers to mask strings that change on every test */ http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index 02613f2..ad5049a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -180,7 +180,7 @@ public class SparkTask extends Task { killJob(); } else if (rc == 4) { LOG.info("The Spark job or one stage of it has too many tasks" + -". Cancelling Spark job " + sparkJobID + " with application ID " + jobID ); +". Cancelling Spark job " + sparkJobID + " with application ID " + jobID); killJob(); } @@ -189,12 +189,7 @@ public class SparkTask extends Task { } sparkJobStatus.cleanup(); } catch (Exception e) { - String msg = "Failed to execute Spark task " + getId() + ", with exception '" + Utilities.getNameMessage(e) + "'"; - - // Has to use full name to make sure it does not conflict with - // org.apache.commons.lang.StringUtils - console.printError(msg, "\n" +
hive git commit: HIVE-19764: Add --SORT_QUERY_RESULTS to hive-blobstore/map_join.q.out (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master 7eac7f605 -> 761597fc5 HIVE-19764: Add --SORT_QUERY_RESULTS to hive-blobstore/map_join.q.out (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/761597fc Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/761597fc Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/761597fc Branch: refs/heads/master Commit: 761597fc55198e5d2bfbb563c40d4db413db0b43 Parents: 7eac7f6 Author: Sahil Takiar Authored: Fri Jun 29 16:01:25 2018 -0700 Committer: Sahil Takiar Committed: Fri Jun 29 16:01:25 2018 -0700 -- .../src/test/queries/clientpositive/map_join.q | 2 + .../test/results/clientpositive/map_join.q.out | 42 ++-- 2 files changed, 23 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/761597fc/itests/hive-blobstore/src/test/queries/clientpositive/map_join.q -- diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/map_join.q b/itests/hive-blobstore/src/test/queries/clientpositive/map_join.q index 63a6c8a..97decba 100644 --- a/itests/hive-blobstore/src/test/queries/clientpositive/map_join.q +++ b/itests/hive-blobstore/src/test/queries/clientpositive/map_join.q @@ -1,5 +1,7 @@ -- Test MAPJOIN hint +-- SORT_QUERY_RESULTS + DROP TABLE keyval1; DROP TABLE keyval2; http://git-wip-us.apache.org/repos/asf/hive/blob/761597fc/itests/hive-blobstore/src/test/results/clientpositive/map_join.q.out -- diff --git a/itests/hive-blobstore/src/test/results/clientpositive/map_join.q.out b/itests/hive-blobstore/src/test/results/clientpositive/map_join.q.out index 268bb52..02ede72 100644 --- a/itests/hive-blobstore/src/test/results/clientpositive/map_join.q.out +++ b/itests/hive-blobstore/src/test/results/clientpositive/map_join.q.out @@ -72,14 +72,19 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@keyval1 POSTHOOK: Input: default@keyval2 A masked pattern was here -99 val_100 val_100 -99 val_100 val_100 +1 val_2 val_2 103val_103 val_103 103val_103 val_103 105val_105 val_105 105val_105 val_105 105val_105 val_105 11 val_11 val_11 +11 val_12 val_12 +11 val_12 val_12 +11 val_12 val_12 +11 val_12 val_12 +11 val_12 val_12 +11 val_12 val_12 111val_111 val_111 117val_118 val_118 117val_118 val_118 @@ -94,12 +99,6 @@ POSTHOOK: Input: default@keyval2 119val_119 val_119 119val_119 val_119 119val_119 val_119 -11 val_12 val_12 -11 val_12 val_12 -11 val_12 val_12 -11 val_12 val_12 -11 val_12 val_12 -11 val_12 val_12 119val_120 val_120 119val_120 val_120 119val_120 val_120 @@ -172,6 +171,7 @@ POSTHOOK: Input: default@keyval2 183val_183 val_183 185val_186 val_186 189val_190 val_190 +19 val_20 val_20 191val_192 val_192 193val_193 val_193 193val_193 val_193 @@ -182,8 +182,6 @@ POSTHOOK: Input: default@keyval2 197val_197 val_197 197val_197 val_197 197val_197 val_197 -1 val_2 val_2 -19 val_20 val_20 199val_200 val_200 199val_200 val_200 205val_205 val_205 @@ -208,10 +206,10 @@ POSTHOOK: Input: default@keyval2 227val_228 val_228 229val_229 val_229 229val_229 val_229 -239val_239 val_239 -239val_239 val_239 23 val_24 val_24 23 val_24 val_24 +239val_239 val_239 +239val_239 val_239 241val_241 val_241 241val_242 val_242 241val_242 val_242 @@ -270,12 +268,13 @@ POSTHOOK: Input: default@keyval2 287val_288 val_288 287val_288 val_288 287val_288 val_288 +29 val_30 val_30 +29 val_30 val_30 291val_292 val_292 291val_292 val_292 295val_296 val_296 295val_296 val_296 -29 val_30 val_30 -29 val_30 val_30 +3 val_4 val_4 305val_305 val_305 305val_306 val_306 307val_307 val_307 @@ -305,6 +304,7 @@ POSTHOOK: Input: default@keyval2 327val_327 val_327 327val_327 val_327 33 val_33 val_33 +33 val_34 val_34 331val_331 val_331 331val_331 val_331 331val_332 val_332 @@ -314,7 +314,6 @@ POSTHOOK: Input: default@keyval2 335val_336 val_336 337val_338 val_338 339val_339 val_339 -33 val_34 val_34 341val_341 val_341 341val_342 val_342 341val_342 val_342 @@ -370,7 +369,6 @@ POSTHOOK: Input: default@keyval2 395val_396 val_396 399val_399 val_399 399val_399 val_399 -3 val_4 val_4 399val_400 val_400 399val_400 val_400 401val_402 val_402 @@ -411,15 +409,15 @@ POSTHOOK: Input: default@keyval2 427
hive git commit: HIVE-19786: RpcServer cancelTask log message is incorrect (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 8c0767625 -> 4810511d6 HIVE-19786: RpcServer cancelTask log message is incorrect (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4810511d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4810511d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4810511d Branch: refs/heads/master Commit: 4810511d6e2b4377b20d70122788d5ad300d8df1 Parents: 8c07676 Author: Bharathkrishna Guruvayoor Murali Authored: Mon Jun 18 10:17:11 2018 -0500 Committer: Sahil Takiar Committed: Mon Jun 18 10:17:11 2018 -0500 -- .../src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/4810511d/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java index f1383d6..babcb54 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java @@ -101,7 +101,8 @@ public class RpcServer implements Closeable { Runnable cancelTask = new Runnable() { @Override public void run() { - LOG.warn("Timed out waiting for test message from Remote Spark driver."); + LOG.warn("Timed out waiting for the completion of SASL negotiation " + + "between HiveServer2 and the Remote Spark Driver."); newRpc.close(); } };
hive git commit: HIVE-19787: Log message when spark-submit has completed (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master c89cf6d5d -> 8c0767625 HIVE-19787: Log message when spark-submit has completed (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8c076762 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8c076762 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8c076762 Branch: refs/heads/master Commit: 8c0767625069418871194f418b99bce8cca1007b Parents: c89cf6d Author: Bharathkrishna Guruvayoor Murali Authored: Mon Jun 18 10:12:10 2018 -0500 Committer: Sahil Takiar Committed: Mon Jun 18 10:12:10 2018 -0500 -- .../java/org/apache/hive/spark/client/SparkSubmitSparkClient.java | 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/8c076762/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java index 1a524b9..31e89b8 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java @@ -211,6 +211,8 @@ class SparkSubmitSparkClient extends AbstractSparkClient { LOG.warn("Child process exited with code {}", exitCode); rpcServer.cancelClient(clientId, "Child process (spark-submit) exited before connecting back with error log " + errStr.toString()); +} else { + LOG.info("Child process (spark-submit) exited successfully."); } } catch (InterruptedException ie) { LOG.warn("Thread waiting on the child process (spark-submit) is interrupted, killing the child process.");
hive git commit: HIVE-19602: Refactor inplace progress code in Hive-on-spark progress monitor to use ProgressMonitor instance (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar, Rui Li)
Repository: hive Updated Branches: refs/heads/master 3a6ad2661 -> c89cf6d5d HIVE-19602: Refactor inplace progress code in Hive-on-spark progress monitor to use ProgressMonitor instance (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar, Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c89cf6d5 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c89cf6d5 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c89cf6d5 Branch: refs/heads/master Commit: c89cf6d5de0343493dc629a0073b5c8e88359a6e Parents: 3a6ad26 Author: Bharathkrishna Guruvayoor Murali Authored: Mon Jun 18 10:03:01 2018 -0500 Committer: Sahil Takiar Committed: Mon Jun 18 10:03:01 2018 -0500 -- .../ql/exec/spark/status/SparkJobMonitor.java | 166 +-- .../exec/spark/status/SparkProgressMonitor.java | 155 + 2 files changed, 160 insertions(+), 161 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/c89cf6d5/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java index e78b1cd..3531ac2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java @@ -22,13 +22,9 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.common.log.InPlaceUpdate; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.session.SessionState; -import org.fusesource.jansi.Ansi; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.PrintStream; -import java.text.DecimalFormat; -import java.text.NumberFormat; import java.text.SimpleDateFormat; import java.util.Date; import java.util.HashSet; @@ -38,8 +34,6 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.TimeUnit; -import static org.fusesource.jansi.Ansi.ansi; - abstract class SparkJobMonitor { protected static final String CLASS_NAME = SparkJobMonitor.class.getName(); @@ -48,6 +42,7 @@ abstract class SparkJobMonitor { protected final PerfLogger perfLogger = SessionState.getPerfLogger(); protected final int checkInterval = 1000; protected final long monitorTimeoutInterval; + private final InPlaceUpdate inPlaceUpdateFn; private final Set completed = new HashSet(); private final int printInterval = 3000; @@ -61,94 +56,20 @@ abstract class SparkJobMonitor { FINISHED } - // in-place progress update related variables protected final boolean inPlaceUpdate; - private int lines = 0; - private final PrintStream out; - - private static final int COLUMN_1_WIDTH = 16; - private static final String HEADER_FORMAT = "%16s%10s %13s %5s %9s %7s %7s %6s "; - private static final String STAGE_FORMAT = "%-16s%10s %13s %5s %9s %7s %7s %6s "; - private static final String HEADER = String.format(HEADER_FORMAT, - "STAGES", "ATTEMPT", "STATUS", "TOTAL", "COMPLETED", "RUNNING", "PENDING", "FAILED"); - private static final int SEPARATOR_WIDTH = 86; - private static final String SEPARATOR = new String(new char[SEPARATOR_WIDTH]).replace("\0", "-"); - private static final String FOOTER_FORMAT = "%-15s %-30s %-4s %-25s"; - private static final int progressBarChars = 30; - - private final NumberFormat secondsFormat = new DecimalFormat("#0.00"); protected SparkJobMonitor(HiveConf hiveConf) { monitorTimeoutInterval = hiveConf.getTimeVar( HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT, TimeUnit.SECONDS); inPlaceUpdate = InPlaceUpdate.canRenderInPlace(hiveConf) && !SessionState.getConsole().getIsSilent(); console = new SessionState.LogHelper(LOG); -out = SessionState.LogHelper.getInfoStream(); +inPlaceUpdateFn = new InPlaceUpdate(SessionState.LogHelper.getInfoStream()); } public abstract int startMonitor(); private void printStatusInPlace(Map progressMap) { - -StringBuilder reportBuffer = new StringBuilder(); - -// Num of total and completed tasks -int sumTotal = 0; -int sumComplete = 0; - -// position the cursor to line 0 -repositionCursor(); - -// header -reprintLine(SEPARATOR); -reprintLineWithColorAsBold(HEADER, Ansi.Color.CYAN); -reprintLine(SEPARATOR); - -SortedSet keys = new TreeSet(progressMap.keySet()); -int idx = 0; -final int numKey = keys.size(); -for (SparkStage stage : keys) { - SparkStageProgress progress = progressMap.get(stage); - final int complete = progress.getSucceededTaskCount(); -
hive git commit: HIVE-18690: Integrate with Spark OutputMetrics (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 473dd0462 -> f04eba3ca HIVE-18690: Integrate with Spark OutputMetrics (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f04eba3c Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f04eba3c Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f04eba3c Branch: refs/heads/master Commit: f04eba3cac3007bfe61ec2f57c92d404f4d40b4c Parents: 473dd04 Author: Sahil Takiar Authored: Tue Feb 27 20:21:55 2018 -0800 Committer: Sahil Takiar Committed: Thu Jun 7 14:29:59 2018 -0500 -- .../hive/ql/exec/spark/TestSparkStatistics.java | 2 +- .../hadoop/hive/ql/exec/FileSinkOperator.java | 16 - .../hive/ql/exec/spark/SparkMetricUtils.java| 63 .../spark/Statistic/SparkStatisticsNames.java | 3 + .../spark/status/impl/SparkMetricsUtils.java| 4 ++ .../hive/spark/client/MetricsCollection.java| 20 ++- .../hive/spark/client/metrics/Metrics.java | 16 - .../spark/client/metrics/OutputMetrics.java | 57 ++ .../spark/client/TestMetricsCollection.java | 14 +++-- 9 files changed, 183 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/f04eba3c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java index f6c5b17..d383873 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java @@ -81,7 +81,7 @@ public class TestSparkStatistics { List sparkStats = Lists.newArrayList(sparkTask.getSparkStatistics() .getStatisticGroup(SparkStatisticsNames.SPARK_GROUP_NAME).getStatistics()); - Assert.assertEquals(24, sparkStats.size()); + Assert.assertEquals(26, sparkStats.size()); Map statsMap = sparkStats.stream().collect( Collectors.toMap(SparkStatistic::getName, SparkStatistic::getValue)); http://git-wip-us.apache.org/repos/asf/hive/blob/f04eba3c/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 83b53f4..c2319bb 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 @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.function.BiFunction; import com.google.common.collect.Lists; @@ -37,6 +38,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; + import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; @@ -46,6 +48,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.Utilities.MissingBucketsContext; +import org.apache.hadoop.hive.ql.exec.spark.SparkMetricUtils; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.io.BucketCodec; import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; @@ -79,16 +82,18 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspecto import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyShim; import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyValue; import org.apache.hadoop.hive.shims.ShimLoader; + import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.ReflectionUtils; + import org.apache.hive.common.util.HiveStringUtils; -import org.apache.hive.common.util.Murmur3; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.function.BiFunction; + /** * File Sink operator implementation. **/ @@ -1228,6 +1233,10 @@ public class FileSinkOperator extends TerminalOperator implements row_count.set(numRows); LOG.info(toString() + ": records written - " + numRows); +if
hive git commit: HIVE-19788: Flaky test: HCatalog-PigServer tests (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 99cb7c005 -> 77c145043 HIVE-19788: Flaky test: HCatalog-PigServer tests (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/77c14504 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/77c14504 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/77c14504 Branch: refs/heads/master Commit: 77c1450430d8ed7b14d7c3c36794a0467c9571db Parents: 99cb7c0 Author: Sahil Takiar Authored: Mon Jun 4 17:03:39 2018 -0500 Committer: Sahil Takiar Committed: Wed Jun 6 15:58:49 2018 -0500 -- .../hive/hcatalog/mapreduce/HCatBaseTest.java | 8 +++-- .../hcatalog/mapreduce/TestPassProperties.java | 4 --- .../hcatalog/pig/AbstractHCatLoaderTest.java| 5 ++-- .../hcatalog/pig/AbstractHCatStorerTest.java| 31 ++-- .../hive/hcatalog/pig/TestE2EScenarios.java | 4 +-- .../pig/TestHCatLoaderComplexSchema.java| 4 +-- .../hcatalog/pig/TestHCatLoaderEncryption.java | 6 ++-- .../hive/hcatalog/pig/TestHCatLoaderStorer.java | 5 ++-- .../hive/hcatalog/pig/TestHCatStorerMulti.java | 9 +++--- .../hcatalog/pig/TestHCatStorerWrapper.java | 3 +- .../hbase/TestPigHBaseStorageHandler.java | 8 ++--- .../mapreduce/TestHCatHiveCompatibility.java| 5 ++-- .../TestHCatHiveThriftCompatibility.java| 3 +- .../mapreduce/TestSequenceFileReadWrite.java| 2 +- 14 files changed, 45 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/77c14504/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java -- diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java index 8471e5b..a304e49 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java @@ -109,14 +109,18 @@ public abstract class HCatBaseTest { server.registerQuery(query, lineNumber); } + public static PigServer createPigServer(boolean stopOnFailure) throws ExecException { +return createPigServer(stopOnFailure, new Properties()); + } + /** * creates PigServer in LOCAL mode. * http://pig.apache.org/docs/r0.12.0/perf.html#error-handling * @param stopOnFailure equivalent of "-stop_on_failure" command line arg, setting to 'true' makes * debugging easier */ - public static PigServer createPigServer(boolean stopOnFailure) throws ExecException { -Properties p = new Properties(); + public static PigServer createPigServer(boolean stopOnFailure, Properties p) throws + ExecException { Path workDir = new Path(System.getProperty("test.tmp.dir", "target" + File.separator + "test" + File.separator + "tmp")); String testId = "HCatBaseTest_" + System.currentTimeMillis(); http://git-wip-us.apache.org/repos/asf/hive/blob/77c14504/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java -- diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java index 1df3c24..332c240 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java @@ -44,8 +44,6 @@ import org.apache.hive.hcatalog.common.HCatException; import org.apache.hive.hcatalog.data.DefaultHCatRecord; import org.apache.hive.hcatalog.data.schema.HCatFieldSchema; import org.apache.hive.hcatalog.data.schema.HCatSchema; -import org.apache.pig.ExecType; -import org.apache.pig.PigServer; import org.junit.Test; public class TestPassProperties { @@ -55,7 +53,6 @@ public class TestPassProperties { private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; private static IDriver driver; - private static PigServer server; private static String[] input; private static HiveConf hiveConf; @@ -81,7 +78,6 @@ public class TestPassProperties { input[i] = i + "," + col1 + "," + col2; } HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); -server = new PigServer(ExecType.LOCAL); } @Test http://git-wip-us.apache.org/repos/asf/hive/blob/77c14504/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/AbstractHCatLoaderTest.java
hive git commit: HIVE-19508: SparkJobMonitor getReport doesn't print stage progress in order (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master f27c38ff5 -> 764b978fd HIVE-19508: SparkJobMonitor getReport doesn't print stage progress in order (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/764b978f Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/764b978f Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/764b978f Branch: refs/heads/master Commit: 764b978fd5802887cdad02ce8074ddf5f8d8e2e4 Parents: f27c38f Author: Bharathkrishna Guruvayoor Murali Authored: Wed Jun 6 14:15:45 2018 -0500 Committer: Sahil Takiar Committed: Wed Jun 6 14:15:45 2018 -0500 -- .../hadoop/hive/ql/exec/spark/SparkTask.java| 5 +- .../exec/spark/status/LocalSparkJobMonitor.java | 4 +- .../spark/status/RemoteSparkJobMonitor.java | 6 +- .../ql/exec/spark/status/SparkJobMonitor.java | 38 - .../ql/exec/spark/status/SparkJobStatus.java| 2 +- .../hive/ql/exec/spark/status/SparkStage.java | 72 .../spark/status/impl/LocalSparkJobStatus.java | 11 ++- .../spark/status/impl/RemoteSparkJobStatus.java | 9 +- .../exec/spark/status/TestSparkJobMonitor.java | 88 9 files changed, 198 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/764b978f/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index ddbb6ba..02613f2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; import org.apache.hadoop.hive.ql.exec.spark.status.impl.SparkMetricsUtils; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkStage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -545,11 +546,11 @@ public class SparkTask extends Task { stageIds.add(stageId); } } - Map progressMap = sparkJobStatus.getSparkStageProgress(); + Map progressMap = sparkJobStatus.getSparkStageProgress(); int sumTotal = 0; int sumComplete = 0; int sumFailed = 0; - for (String s : progressMap.keySet()) { + for (SparkStage s : progressMap.keySet()) { SparkStageProgress progress = progressMap.get(s); final int complete = progress.getSucceededTaskCount(); final int total = progress.getTotalTaskCount(); http://git-wip-us.apache.org/repos/asf/hive/blob/764b978f/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java index 4ce9f53..2a6c33b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java @@ -42,7 +42,7 @@ public class LocalSparkJobMonitor extends SparkJobMonitor { boolean done = false; int rc = 0; JobExecutionStatus lastState = null; -Map lastProgressMap = null; +Map lastProgressMap = null; perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); @@ -68,7 +68,7 @@ public class LocalSparkJobMonitor extends SparkJobMonitor { } } else if (state != lastState || state == JobExecutionStatus.RUNNING) { lastState = state; - Map progressMap = sparkJobStatus.getSparkStageProgress(); + Map progressMap = sparkJobStatus.getSparkStageProgress(); switch (state) { case RUNNING: http://git-wip-us.apache.org/repos/asf/hive/blob/764b978f/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index 98c228b..004b50b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -56,7 +56,7 @@ public class RemoteSparkJobMonitor extends
hive git commit: HIVE-19525: Spark task logs print PLAN PATH excessive number of times (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 54b4b2d42 -> f27c38ff5 HIVE-19525: Spark task logs print PLAN PATH excessive number of times (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f27c38ff Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f27c38ff Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f27c38ff Branch: refs/heads/master Commit: f27c38ff55902827499192a4f8cf8ed37d6fd967 Parents: 54b4b2d Author: Bharathkrishna Guruvayoor Murali Authored: Wed Jun 6 14:12:47 2018 -0500 Committer: Sahil Takiar Committed: Wed Jun 6 14:12:47 2018 -0500 -- .../apache/hadoop/hive/ql/exec/Utilities.java | 124 +-- 1 file changed, 62 insertions(+), 62 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/f27c38ff/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 2177c33..80478ca 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -408,7 +408,19 @@ public final class Utilities { * @throws RuntimeException if the configuration files are not proper or if plan can not be loaded */ private static BaseWork getBaseWork(Configuration conf, String name) { -Path path = null; + +Path path = getPlanPath(conf, name); +LOG.debug("PLAN PATH = {}", path); +if (path == null) { // Map/reduce plan may not be generated + return null; +} + +BaseWork gWork = gWorkMap.get(conf).get(path); +if (gWork != null) { + LOG.debug("Found plan in cache for name: {}", name); + return gWork; +} + InputStream in = null; Kryo kryo = SerializationUtilities.borrowKryo(); try { @@ -424,73 +436,61 @@ public final class Utilities { kryo.setClassLoader(newLoader); } } - - path = getPlanPath(conf, name); - LOG.info("PLAN PATH = {}", path); - if (path == null) { // Map/reduce plan may not be generated -return null; + Path localPath = path; + LOG.debug("local path = {}", localPath); + final long serializedSize; + final String planMode; + if (HiveConf.getBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN)) { +String planStringPath = path.toUri().getPath(); +LOG.debug("Loading plan from string: {}", planStringPath); +String planString = conf.getRaw(planStringPath); +if (planString == null) { + LOG.info("Could not find plan string in conf"); + return null; +} +serializedSize = planString.length(); +planMode = "RPC"; +byte[] planBytes = Base64.decodeBase64(planString); +in = new ByteArrayInputStream(planBytes); +in = new InflaterInputStream(in); + } else { +LOG.debug("Open file to read in plan: {}", localPath); +FileSystem fs = localPath.getFileSystem(conf); +in = fs.open(localPath); +serializedSize = fs.getFileStatus(localPath).getLen(); +planMode = "FILE"; } - BaseWork gWork = gWorkMap.get(conf).get(path); - if (gWork == null) { -Path localPath = path; -LOG.debug("local path = {}", localPath); -final long serializedSize; -final String planMode; -if (HiveConf.getBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN)) { - String planStringPath = path.toUri().getPath(); - LOG.debug("Loading plan from string: {}", planStringPath); - String planString = conf.getRaw(planStringPath); - if (planString == null) { -LOG.info("Could not find plan string in conf"); -return null; - } - serializedSize = planString.length(); - planMode = "RPC"; - byte[] planBytes = Base64.decodeBase64(planString); - in = new ByteArrayInputStream(planBytes); - in = new InflaterInputStream(in); + if(MAP_PLAN_NAME.equals(name)){ +if (ExecMapper.class.getName().equals(conf.get(MAPRED_MAPPER_CLASS))){ + gWork = SerializationUtilities.deserializePlan(kryo, in, MapWork.class); +} else if(MergeFileMapper.class.getName().equals(conf.get(MAPRED_MAPPER_CLASS))) { + gWork = SerializationUtilities.deserializePlan(kryo, in, MergeFileWork.class); +} else if(ColumnTruncateMapper.class.getName().equals(conf.get(MAPRED_MAPPER_CLASS))) { + gWork = SerializationUtilities.deserializePlan(kryo, in, ColumnTruncateWork.class); } else { - LOG.debug("Open file to read in plan: {}",
hive git commit: HIVE-19079: Add extended query string to Spark job description (Sahil Takiar, reviewed by Aihua Xu) (addendum)
Repository: hive Updated Branches: refs/heads/master 8e1e537bf -> 9615f24b9 HIVE-19079: Add extended query string to Spark job description (Sahil Takiar, reviewed by Aihua Xu) (addendum) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9615f24b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9615f24b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9615f24b Branch: refs/heads/master Commit: 9615f24b9c0baf68de3b24ed6366a277d44564cb Parents: 8e1e537 Author: Sahil Takiar Authored: Wed Jun 6 12:55:46 2018 -0500 Committer: Sahil Takiar Committed: Wed Jun 6 12:55:46 2018 -0500 -- .../hive/ql/exec/spark/TestHiveSparkClient.java | 18 ++ 1 file changed, 18 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/9615f24b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveSparkClient.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveSparkClient.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveSparkClient.java index 239c098..b960508 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveSparkClient.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveSparkClient.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.hive.ql.exec.spark; import org.apache.hadoop.fs.FileSystem;
hive git commit: HIVE-18533: Add option to use InProcessLauncher to submit spark jobs (Sahil Takiar, reviewed by Rui Li) (addendum)
Repository: hive Updated Branches: refs/heads/master 13fbae573 -> 8e1e537bf HIVE-18533: Add option to use InProcessLauncher to submit spark jobs (Sahil Takiar, reviewed by Rui Li) (addendum) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8e1e537b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8e1e537b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8e1e537b Branch: refs/heads/master Commit: 8e1e537bf9f8204fd27e806af0f62aed980bf36a Parents: 13fbae5 Author: Sahil Takiar Authored: Wed Jun 6 12:48:57 2018 -0500 Committer: Sahil Takiar Committed: Wed Jun 6 12:48:57 2018 -0500 -- .../spark/client/SparkLauncherSparkClient.java | 2 +- .../spark/client/TestSparkLauncherSparkClient.java | 17 + 2 files changed, 18 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/8e1e537b/spark-client/src/main/java/org/apache/hive/spark/client/SparkLauncherSparkClient.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkLauncherSparkClient.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkLauncherSparkClient.java index cf52c4f..d45b77f 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkLauncherSparkClient.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkLauncherSparkClient.java @@ -61,7 +61,7 @@ public class SparkLauncherSparkClient extends AbstractSparkClient { SparkAppHandle.State.KILLED, SparkAppHandle.State.LOST); - private AbstractLauncher sparkLauncher; + private transient AbstractLauncher sparkLauncher; SparkLauncherSparkClient(RpcServer rpcServer, Map conf, http://git-wip-us.apache.org/repos/asf/hive/blob/8e1e537b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java -- diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java index 8434fa9..986215e 100644 --- a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java +++ b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hive.spark.client; import org.apache.hive.spark.client.rpc.RpcServer;
hive git commit: HIVE-19079: Add extended query string to Spark job description (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 3dcd36336 -> 98982e619 HIVE-19079: Add extended query string to Spark job description (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/98982e61 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/98982e61 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/98982e61 Branch: refs/heads/master Commit: 98982e6193ce05111d031bb8d54ce530ae41853f Parents: 3dcd363 Author: Sahil Takiar Authored: Mon Apr 2 09:31:38 2018 -0700 Committer: Sahil Takiar Committed: Tue Jun 5 12:00:34 2018 -0500 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 2 + .../java/org/apache/hadoop/hive/ql/Driver.java | 8 +- .../ql/exec/spark/LocalHiveSparkClient.java | 3 + .../ql/exec/spark/RemoteHiveSparkClient.java| 7 +- .../hive/ql/exec/spark/TestHiveSparkClient.java | 97 5 files changed, 115 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/98982e61/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 9004894..cd425aa 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1494,6 +1494,8 @@ public class HiveConf extends Configuration { HIVEQUERYID("hive.query.id", "", "ID for query being executed (might be multiple per a session)"), +HIVESPARKJOBNAMELENGTH("hive.spark.jobname.length", 10, "max jobname length for Hive on " + +"Spark queries"), HIVEJOBNAMELENGTH("hive.jobname.length", 50, "max jobname length"), // hive jar http://git-wip-us.apache.org/repos/asf/hive/blob/98982e61/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 7d5e7d4..a3dcc3b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -2033,7 +2033,13 @@ public class Driver implements IDriver { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DRIVER_EXECUTE); boolean noName = StringUtils.isEmpty(conf.get(MRJobConfig.JOB_NAME)); -int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); + +int maxlen; +if ("spark".equals(conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) { + maxlen = conf.getIntVar(HiveConf.ConfVars.HIVESPARKJOBNAMELENGTH); +} else { + maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); +} Metrics metrics = MetricsFactory.getInstance(); String queryId = queryState.getQueryId(); http://git-wip-us.apache.org/repos/asf/hive/blob/98982e61/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index dff3b0f..72ff53e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -163,6 +163,9 @@ public class LocalHiveSparkClient implements HiveSparkClient { // Execute generated plan. JavaPairRDD finalRDD = plan.generateGraph(); +// We get the query name for this SparkTask and set it to the description for the associated +// Spark job; query names are guaranteed to be unique for each Spark job because the task id +// is concatenated to the end of the query name sc.setJobGroup("queryId = " + sparkWork.getQueryId(), DagUtils.getQueryName(jobConf)); // We use Spark RDD async action to submit job as it's the only way to get jobId now. http://git-wip-us.apache.org/repos/asf/hive/blob/98982e61/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index e8f39ae..d31a202 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hive.ql.exec.spark; +import com.google.common.annotations.VisibleForTesting; import
hive git commit: HIVE-19752: PerfLogger integration for critical Hive-on-S3 paths (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master da6638666 -> 1b7f62b05 HIVE-19752: PerfLogger integration for critical Hive-on-S3 paths (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1b7f62b0 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1b7f62b0 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1b7f62b0 Branch: refs/heads/master Commit: 1b7f62b05daef3992d8637dcf09d0037177f7527 Parents: da66386 Author: Sahil Takiar Authored: Thu May 31 16:51:33 2018 -0500 Committer: Sahil Takiar Committed: Tue Jun 5 08:24:36 2018 -0500 -- .../apache/hadoop/hive/ql/log/PerfLogger.java | 6 + .../apache/hadoop/hive/ql/exec/MoveTask.java| 7 ++ .../apache/hadoop/hive/ql/exec/Utilities.java | 5 .../apache/hadoop/hive/ql/metadata/Hive.java| 26 +--- 4 files changed, 41 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1b7f62b0/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java -- diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java index 3d6315c..111e614 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java +++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java @@ -46,6 +46,7 @@ public class PerfLogger { public static final String DO_AUTHORIZATION = "doAuthorization"; public static final String DRIVER_EXECUTE = "Driver.execute"; public static final String INPUT_SUMMARY = "getInputSummary"; + public static final String INPUT_PATHS = "getInputPaths"; public static final String GET_SPLITS = "getSplits"; public static final String RUN_TASKS = "runTasks"; public static final String SERIALIZE_PLAN = "serializePlan"; @@ -85,6 +86,11 @@ public class PerfLogger { public static final String SPARK_OPTIMIZE_TASK_TREE = "SparkOptimizeTaskTree"; public static final String SPARK_FLUSH_HASHTABLE = "SparkFlushHashTable."; + public static final String FILE_MOVES = "FileMoves"; + public static final String LOAD_TABLE = "LoadTable"; + public static final String LOAD_PARTITION = "LoadPartition"; + public static final String LOAD_DYNAMIC_PARTITIONS = "LoadDynamicPartitions"; + protected final Map startTimes = new HashMap(); protected final Map endTimes = new HashMap(); http://git-wip-us.apache.org/repos/asf/hive/blob/1b7f62b0/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index dbda5fd..f80a945 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveLock; import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; import org.apache.hadoop.hive.ql.lockmgr.LockException; +import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; @@ -60,6 +61,7 @@ import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.api.StageType; +import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,6 +90,9 @@ public class MoveTask extends Task implements Serializable { private void moveFile(Path sourcePath, Path targetPath, boolean isDfsDir) throws HiveException { try { + PerfLogger perfLogger = SessionState.getPerfLogger(); + perfLogger.PerfLogBegin("MoveTask", PerfLogger.FILE_MOVES); + String mesg = "Moving data to " + (isDfsDir ? "" : "local ") + "directory " + targetPath.toString(); String mesg_detail = " from " + sourcePath.toString(); @@ -101,6 +106,8 @@ public class MoveTask extends Task implements Serializable { FileSystem dstFs = FileSystem.getLocal(conf); moveFileFromDfsToLocal(sourcePath, targetPath, fs, dstFs); } + + perfLogger.PerfLogEnd("MoveTask", PerfLogger.FILE_MOVES); } catch (Exception e) { throw new HiveException("Unable to move source " + sourcePath + " to destination " + targetPath, e);
[1/2] hive git commit: HIVE-18533: Add option to use InProcessLauncher to submit spark jobs (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master f80cff9ab -> da6638666 http://git-wip-us.apache.org/repos/asf/hive/blob/da663866/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java -- diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java new file mode 100644 index 000..8434fa9 --- /dev/null +++ b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkLauncherSparkClient.java @@ -0,0 +1,77 @@ +package org.apache.hive.spark.client; + +import org.apache.hive.spark.client.rpc.RpcServer; +import org.apache.spark.launcher.SparkAppHandle; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestSparkLauncherSparkClient { + + @Test + public void testSparkLauncherFutureGet() { +testChainOfStates(SparkAppHandle.State.CONNECTED, SparkAppHandle.State.SUBMITTED, +SparkAppHandle.State.RUNNING); +testChainOfStates(SparkAppHandle.State.CONNECTED, SparkAppHandle.State.SUBMITTED, +SparkAppHandle.State.FINISHED); +testChainOfStates(SparkAppHandle.State.CONNECTED, SparkAppHandle.State.SUBMITTED, +SparkAppHandle.State.FAILED); +testChainOfStates(SparkAppHandle.State.CONNECTED, SparkAppHandle.State.SUBMITTED, +SparkAppHandle.State.KILLED); + +testChainOfStates(SparkAppHandle.State.LOST); +testChainOfStates(SparkAppHandle.State.CONNECTED, SparkAppHandle.State.LOST); +testChainOfStates(SparkAppHandle.State.CONNECTED, SparkAppHandle.State.SUBMITTED, +SparkAppHandle.State.LOST); + } + + private void testChainOfStates(SparkAppHandle.State... states) { +SparkAppHandle sparkAppHandle = mock(SparkAppHandle.class); +RpcServer rpcServer = mock(RpcServer.class); +String clientId = ""; + +CountDownLatch shutdownLatch = new CountDownLatch(1); + +SparkLauncherSparkClient.SparkAppListener sparkAppListener = new SparkLauncherSparkClient.SparkAppListener( +shutdownLatch, rpcServer, clientId); +Future sparkLauncherFuture = SparkLauncherSparkClient.createSparkLauncherFuture( +shutdownLatch, sparkAppHandle, rpcServer, clientId); + +CompletableFuture future = CompletableFuture.runAsync(() -> { + try { +sparkLauncherFuture.get(); + } catch (InterruptedException | ExecutionException e) { +throw new RuntimeException(e); + } +}); + +for (int i = 0; i < states.length - 1; i++) { + when(sparkAppHandle.getState()).thenReturn(states[i]); + sparkAppListener.stateChanged(sparkAppHandle); + Assert.assertTrue(!future.isDone()); +} + +when(sparkAppHandle.getState()).thenReturn(states[states.length - 1]); +sparkAppListener.stateChanged(sparkAppHandle); +try { + future.get(60, TimeUnit.SECONDS); +} catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException("SparkLauncherFuture failed to complete after transitioning to " + + "state " + states[states.length - 1], e); +} +Assert.assertTrue(future.isDone()); +Assert.assertEquals(shutdownLatch.getCount(), 0); +verify(sparkAppHandle).disconnect(); + } +}
[2/2] hive git commit: HIVE-18533: Add option to use InProcessLauncher to submit spark jobs (Sahil Takiar, reviewed by Rui Li)
HIVE-18533: Add option to use InProcessLauncher to submit spark jobs (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/da663866 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/da663866 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/da663866 Branch: refs/heads/master Commit: da66386662fbbcbde9501b4a7b27d076bcc790d4 Parents: f80cff9 Author: Sahil Takiar Authored: Tue Jun 5 08:00:54 2018 -0500 Committer: Sahil Takiar Committed: Tue Jun 5 08:00:54 2018 -0500 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 12 +- itests/qtest-spark/pom.xml | 6 + .../test/resources/testconfiguration.properties | 3 +- .../clientpositive/spark_in_process_launcher.q | 6 + .../spark/spark_in_process_launcher.q.out | 96 +++ spark-client/pom.xml| 6 + .../hive/spark/client/AbstractSparkClient.java | 600 .../apache/hive/spark/client/JobHandleImpl.java | 8 +- .../apache/hive/spark/client/SparkClient.java | 7 + .../hive/spark/client/SparkClientFactory.java | 16 +- .../hive/spark/client/SparkClientImpl.java | 703 --- .../spark/client/SparkLauncherSparkClient.java | 220 ++ .../spark/client/SparkSubmitSparkClient.java| 237 +++ .../apache/hive/spark/client/TestJobHandle.java | 2 +- .../hive/spark/client/TestSparkClient.java | 25 +- .../client/TestSparkLauncherSparkClient.java| 77 ++ 16 files changed, 1306 insertions(+), 718 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/da663866/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 3295d1d..56d2de0 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -316,6 +316,9 @@ public class HiveConf extends Configuration { public static final String HIVE_SERVER2_AUTHENTICATION_LDAP_USERMEMBERSHIPKEY_NAME = "hive.server2.authentication.ldap.userMembershipKey"; + public static final String HIVE_SPARK_SUBMIT_CLIENT = "spark-submit"; + public static final String HIVE_SPARK_LAUNCHER_CLIENT = "spark-launcher"; + /** * dbVars are the parameters can be set per database. If these * parameters are set as a database property, when switching to that @@ -4245,6 +4248,11 @@ public class HiveConf extends Configuration { "If a Spark job contains more tasks than the maximum, it will be cancelled. A value of -1 means no limit."), SPARK_STAGE_MAX_TASKS("hive.spark.stage.max.tasks", -1, "The maximum number of tasks a stage in a Spark job may have.\n" + "If a Spark job stage contains more tasks than the maximum, the job will be cancelled. A value of -1 means no limit."), +SPARK_CLIENT_TYPE("hive.spark.client.type", HIVE_SPARK_SUBMIT_CLIENT, +"Controls how the Spark application is launched. If " + HIVE_SPARK_SUBMIT_CLIENT + " is " + +"specified (default) then the spark-submit shell script is used to launch the Spark " + +"app. If " + HIVE_SPARK_LAUNCHER_CLIENT + " is specified then Spark's " + +"InProcessLauncher is used to programmatically launch the app."), 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, @@ -4335,7 +4343,8 @@ public class HiveConf extends Configuration { "Comma separated list of variables which are used internally and should not be configurable."), HIVE_SPARK_RSC_CONF_LIST("hive.spark.rsc.conf.list", SPARK_OPTIMIZE_SHUFFLE_SERDE.varname + "," + -SPARK_CLIENT_FUTURE_TIMEOUT.varname, +SPARK_CLIENT_FUTURE_TIMEOUT.varname + "," + +SPARK_CLIENT_TYPE.varname, "Comma separated list of variables which are related to remote spark context.\n" + "Changing these variables will result in re-creating the spark session."), HIVE_QUERY_TIMEOUT_SECONDS("hive.query.timeout.seconds", "0s", @@ -5802,5 +5811,4 @@ public class HiveConf extends Configuration { } return ret; } - } http://git-wip-us.apache.org/repos/asf/hive/blob/da663866/itests/qtest-spark/pom.xml -- diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml index d0e7eb8..8ed3171 100644 --- a/itests/qtest-spark/pom.xml +++ b/itests/qtest-spark/pom.xml @@ -64,6 +64,12 @@ + org.apache.spark +
hive git commit: HIVE-19759: Flaky test: TestRpc#testServerPort (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master d682ca926 -> 2d3faead7 HIVE-19759: Flaky test: TestRpc#testServerPort (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2d3faead Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2d3faead Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2d3faead Branch: refs/heads/master Commit: 2d3faead7f15200b21dd21e7320e0f3853878f71 Parents: d682ca9 Author: Sahil Takiar Authored: Fri Jun 1 08:15:27 2018 -0500 Committer: Sahil Takiar Committed: Mon Jun 4 15:24:19 2018 -0500 -- .../apache/hadoop/hive/common/ServerUtils.java | 8 .../apache/hive/spark/client/rpc/TestRpc.java | 39 2 files changed, 39 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/2d3faead/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java -- diff --git a/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java b/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java index 7979bbe..d7f4b14 100644 --- a/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java +++ b/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java @@ -18,7 +18,9 @@ package org.apache.hadoop.hive.common; +import java.io.IOException; import java.net.InetAddress; +import java.net.ServerSocket; import java.net.UnknownHostException; import org.slf4j.Logger; @@ -77,4 +79,10 @@ public class ServerUtils { } } + public static int findFreePort() throws IOException { +ServerSocket socket= new ServerSocket(0); +int port = socket.getLocalPort(); +socket.close(); +return port; + } } http://git-wip-us.apache.org/repos/asf/hive/blob/2d3faead/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java -- diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java index 5653e4d..013bcff 100644 --- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java +++ b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java @@ -46,6 +46,7 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.Future; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hive.common.ServerUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +63,7 @@ public class TestRpc { private Collection closeables; private static final Map emptyConfig = ImmutableMap.of(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname, "DEBUG"); + private static final int RETRY_ACQUIRE_PORT_COUNT = 10; @Before public void setUp() { @@ -187,10 +189,21 @@ public class TestRpc { assertTrue("Port should be within configured port range:" + server1.getPort(), server1.getPort() >= 49152 && server1.getPort() <= 49333); IOUtils.closeQuietly(server1); -int expectedPort = 65535; -config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort)); -RpcServer server2 = new RpcServer(config); -assertTrue("Port should match configured one: " + server2.getPort(), server2.getPort() == expectedPort); +int expectedPort = ServerUtils.findFreePort(); +RpcServer server2 = null; +for (int i = 0; i < RETRY_ACQUIRE_PORT_COUNT; i++) { + try { +config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort)); +server2 = new RpcServer(config); +break; + } catch (Exception e) { +LOG.debug("Error while connecting to port " + expectedPort + " retrying: " + e.getMessage()); +expectedPort = ServerUtils.findFreePort(); + } +} + +assertNotNull("Unable to create RpcServer with any attempted port", server2); +assertEquals("Port should match configured one: " + server2.getPort(), expectedPort, server2.getPort()); IOUtils.closeQuietly(server2); config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, "49552-49222,49223,49224-49333"); @@ -204,10 +217,20 @@ public class TestRpc { } // Retry logic -expectedPort = 65535; -config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort) + ",21-23"); -RpcServer server3 = new RpcServer(config); -assertTrue("Port should match configured one:" + server3.getPort(), server3.getPort() == expectedPort); +expectedPort = ServerUtils.findFreePort(); +RpcServer server3 = null; +for (int i = 0; i < RETRY_ACQUIRE_PORT_COUNT; i++) { + try { +
hive git commit: HIVE-18652: Print Spark metrics on console (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master 2028749b1 -> d682ca926 HIVE-18652: Print Spark metrics on console (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d682ca92 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d682ca92 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d682ca92 Branch: refs/heads/master Commit: d682ca9266df182e977b35ab47771dbac2ec Parents: 2028749 Author: Sahil Takiar Authored: Mon Jun 4 13:36:04 2018 -0500 Committer: Sahil Takiar Committed: Mon Jun 4 13:36:48 2018 -0500 -- .../hive/ql/exec/spark/TestSparkStatistics.java | 2 +- .../hadoop/hive/ql/exec/spark/SparkTask.java| 83 +++- .../spark/Statistic/SparkStatisticGroup.java| 4 + .../spark/Statistic/SparkStatisticsNames.java | 25 -- .../spark/status/impl/SparkMetricsUtils.java| 37 ++--- .../hive/spark/client/MetricsCollection.java| 20 - .../hive/spark/client/metrics/InputMetrics.java | 12 ++- .../client/metrics/ShuffleReadMetrics.java | 21 - .../client/metrics/ShuffleWriteMetrics.java | 11 ++- .../spark/client/TestMetricsCollection.java | 15 ++-- 10 files changed, 190 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d682ca92/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java index 4413161..f6c5b17 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java @@ -81,7 +81,7 @@ public class TestSparkStatistics { List sparkStats = Lists.newArrayList(sparkTask.getSparkStatistics() .getStatisticGroup(SparkStatisticsNames.SPARK_GROUP_NAME).getStatistics()); - Assert.assertEquals(18, sparkStats.size()); + Assert.assertEquals(24, sparkStats.size()); Map statsMap = sparkStats.stream().collect( Collectors.toMap(SparkStatistic::getName, SparkStatistic::getValue)); http://git-wip-us.apache.org/repos/asf/hive/blob/d682ca92/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index 8038771..ddbb6ba 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -35,6 +35,7 @@ import com.google.common.base.Throwables; import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; +import org.apache.hadoop.hive.ql.exec.spark.status.impl.SparkMetricsUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -161,6 +162,7 @@ public class SparkTask extends Task { if (rc == 0) { sparkStatistics = sparkJobStatus.getSparkStatistics(); +printConsoleMetrics(); printExcessiveGCWarning(); if (LOG.isInfoEnabled() && sparkStatistics != null) { LOG.info(sparkStatisticsToString(sparkStatistics, sparkJobID)); @@ -222,6 +224,79 @@ public class SparkTask extends Task { return rc; } + private void printConsoleMetrics() { +SparkStatisticGroup sparkStatisticGroup = sparkStatistics.getStatisticGroup( +SparkStatisticsNames.SPARK_GROUP_NAME); + +if (sparkStatisticGroup != null) { + String colon = ": "; + String forwardSlash = " / "; + String separator = ", "; + + String metricsString = String.format("Spark Job[%d] Metrics: ", sparkJobID); + + // Task Duration Time + if (sparkStatisticGroup.containsSparkStatistic(SparkStatisticsNames.TASK_DURATION_TIME)) { +metricsString += SparkStatisticsNames.TASK_DURATION_TIME + colon + +SparkMetricsUtils.getSparkStatisticAsLong(sparkStatisticGroup, +SparkStatisticsNames.TASK_DURATION_TIME) + separator; + } + + // Executor CPU Time + if (sparkStatisticGroup.containsSparkStatistic(SparkStatisticsNames.EXECUTOR_CPU_TIME)) { +metricsString += SparkStatisticsNames.EXECUTOR_CPU_TIME + colon + +SparkMetricsUtils.getSparkStatisticAsLong(sparkStatisticGroup, +
hive git commit: HIVE-19370: Issue: ADD Months function on timestamp datatype fields in hive (Bharathkrishna Guruvayoor Murali, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master da13a1379 -> 28779d202 HIVE-19370: Issue: ADD Months function on timestamp datatype fields in hive (Bharathkrishna Guruvayoor Murali, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/28779d20 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/28779d20 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/28779d20 Branch: refs/heads/master Commit: 28779d202f085531ca547988f0999028f3903822 Parents: da13a13 Author: Bharathkrishna Guruvayoor Murali Authored: Fri Jun 1 05:51:35 2018 -0500 Committer: Sahil Takiar Committed: Fri Jun 1 05:51:35 2018 -0500 -- .../ql/udf/generic/GenericUDFAddMonths.java | 73 ++--- .../ql/udf/generic/TestGenericUDFAddMonths.java | 109 +++ .../queries/clientpositive/udf_add_months.q | 2 + .../results/clientpositive/udf_add_months.q.out | 20 +++- 4 files changed, 181 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/28779d20/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAddMonths.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAddMonths.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAddMonths.java index dae4b97..ea1544f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAddMonths.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAddMonths.java @@ -22,11 +22,13 @@ import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveO import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping.STRING_GROUP; import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping.VOID_GROUP; +import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.Date; import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -44,32 +46,61 @@ import org.apache.hive.common.util.DateUtils; * */ @Description(name = "add_months", -value = "_FUNC_(start_date, num_months) - Returns the date that is num_months after start_date.", -extended = "start_date is a string in the format '-MM-dd HH:mm:ss' or" -+ " '-MM-dd'. num_months is a number. The time part of start_date is " -+ "ignored.\n" -+ "Example:\n " + " > SELECT _FUNC_('2009-08-31', 1) FROM src LIMIT 1;\n" + " '2009-09-30'") +value = "_FUNC_(start_date, num_months, output_date_format) - " ++ "Returns the date that is num_months after start_date.", +extended = "start_date is a string or timestamp indicating a valid date. " ++ "num_months is a number. output_date_format is an optional String which specifies the format for output.\n" ++ "The default output format is '-MM-dd'.\n" ++ "Example:\n > SELECT _FUNC_('2009-08-31', 1) FROM src LIMIT 1;\n" + " '2009-09-30'." ++ "\n > SELECT _FUNC_('2017-12-31 14:15:16', 2, '-MM-dd HH:mm:ss') LIMIT 1;\n" ++ "'2018-02-28 14:15:16'.\n") @NDV(maxNdv = 250) // 250 seems to be reasonable upper limit for this public class GenericUDFAddMonths extends GenericUDF { - private transient Converter[] converters = new Converter[2]; - private transient PrimitiveCategory[] inputTypes = new PrimitiveCategory[2]; - private final Calendar calendar = Calendar.getInstance(); + private transient Converter[] tsConverters = new Converter[3]; + private transient PrimitiveCategory[] tsInputTypes = new PrimitiveCategory[3]; + private transient Converter[] dtConverters = new Converter[3]; + private transient PrimitiveCategory[] dtInputTypes = new PrimitiveCategory[3]; private final Text output = new Text(); + private transient SimpleDateFormat formatter = null; + private final Calendar calendar = Calendar.getInstance(); private transient Integer numMonthsConst; private transient boolean isNumMonthsConst; @Override public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { -checkArgsSize(arguments, 2, 2); +checkArgsSize(arguments, 2, 3); checkArgPrimitive(arguments, 0); checkArgPrimitive(arguments, 1); -checkArgGroups(arguments, 0, inputTypes, STRING_GROUP, DATE_GROUP, VOID_GROUP); -checkArgGroups(arguments, 1, inputTypes,
hive git commit: HIVE-14388: Add number of rows inserted message after insert command in Beeline (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar, Peter Vary)
Repository: hive Updated Branches: refs/heads/master 1c970d924 -> da13a1379 HIVE-14388: Add number of rows inserted message after insert command in Beeline (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar, Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/da13a137 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/da13a137 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/da13a137 Branch: refs/heads/master Commit: da13a13797c7d9926f9a2332ee7b33c7b5f08a1e Parents: 1c970d9 Author: Bharathkrishna Guruvayoor Murali Authored: Fri Jun 1 05:46:54 2018 -0500 Committer: Sahil Takiar Committed: Fri Jun 1 05:46:54 2018 -0500 -- beeline/src/main/resources/BeeLine.properties | 2 +- .../org/apache/hive/jdbc/TestJdbcDriver2.java | 36 --- .../org/apache/hive/jdbc/HiveStatement.java | 16 ++- .../java/org/apache/hadoop/hive/ql/Driver.java | 15 +++ .../org/apache/hadoop/hive/ql/MapRedStats.java | 10 ++ .../org/apache/hadoop/hive/ql/QueryState.java | 12 +++ .../hadoop/hive/ql/exec/FileSinkOperator.java | 23 ++-- .../hive/ql/exec/mr/HadoopJobExecHelper.java| 11 ++ .../clientpositive/llap/dp_counter_mm.q.out | 8 ++ .../clientpositive/llap/dp_counter_non_mm.q.out | 8 ++ .../clientpositive/llap/orc_ppd_basic.q.out | 1 + .../llap/tez_input_counters.q.out | 1 + service-rpc/if/TCLIService.thrift | 1 + .../gen/thrift/gen-cpp/TCLIService_types.cpp| 22 .../src/gen/thrift/gen-cpp/TCLIService_types.h | 12 ++- .../rpc/thrift/TGetOperationStatusResp.java | 107 ++- service-rpc/src/gen/thrift/gen-php/Types.php| 23 .../src/gen/thrift/gen-py/TCLIService/ttypes.py | 15 ++- .../gen/thrift/gen-rb/t_c_l_i_service_types.rb | 4 +- .../org/apache/hive/service/cli/CLIService.java | 2 + .../hive/service/cli/OperationStatus.java | 9 ++ .../hive/service/cli/operation/Operation.java | 4 + .../service/cli/thrift/ThriftCLIService.java| 6 ++ 23 files changed, 303 insertions(+), 45 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/da13a137/beeline/src/main/resources/BeeLine.properties -- diff --git a/beeline/src/main/resources/BeeLine.properties b/beeline/src/main/resources/BeeLine.properties index c41b3ed..30b4ef9 100644 --- a/beeline/src/main/resources/BeeLine.properties +++ b/beeline/src/main/resources/BeeLine.properties @@ -138,7 +138,7 @@ abort-drop-all: Aborting drop all tables. drivers-found-count: 0#No driver classes found|1#{0} driver class found|1<{0} driver classes found rows-selected: 0#No rows selected|1#{0} row selected|1<{0} rows selected -rows-affected: 0#No rows affected|1#{0} row affected|1<{0} rows affected|0>Unknown rows affected +rows-affected: -1#Unknown rows affected|0#No rows affected|1#{0} row affected|1<{0} rows affected active-connections: 0#No active connections|1#{0} active connection:|1<{0} active connections: time-ms: ({0,number,#.###} seconds) http://git-wip-us.apache.org/repos/asf/hive/blob/da13a137/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java -- 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 b217259..d47c136 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 @@ -977,7 +977,7 @@ public class TestJdbcDriver2 { assertNotNull("ResultSet is null", res); assertTrue("getResultSet() not returning expected ResultSet", res == stmt .getResultSet()); -assertEquals("get update count not as expected", -1, stmt.getUpdateCount()); +assertEquals("get update count not as expected", 0, stmt.getUpdateCount()); int i = 0; ResultSetMetaData meta = res.getMetaData(); @@ -2962,23 +2962,25 @@ public class TestJdbcDriver2 { private void testInsertOverwrite(HiveStatement stmt) throws SQLException { String tblName = "testInsertOverwriteExecAsync"; -int rowCount = 0; -stmt.execute("create table " + tblName + " (col1 int , col2 string)"); -boolean isResulSet = -stmt.executeAsync("insert overwrite table " + tblName + " select * from " + tableName); -assertFalse(isResulSet); -// HiveStatement#getUpdateCount blocks until the async query is complete -stmt.getUpdateCount(); -// Read from the new table -ResultSet rs = stmt.executeQuery("select * from " + tblName); -assertNotNull(rs); -while (rs.next()) { - String value =
hive git commit: HIVE-19716: Set spark.local.dir for a few more HoS integration tests (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 47e85b73d -> 068d007b8 HIVE-19716: Set spark.local.dir for a few more HoS integration tests (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/068d007b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/068d007b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/068d007b Branch: refs/heads/master Commit: 068d007b8874f9eecbf30bb2739831ad9114107a Parents: 47e85b7 Author: Sahil Takiar Authored: Fri May 25 11:52:18 2018 -0700 Committer: Sahil Takiar Committed: Tue May 29 12:05:38 2018 -0500 -- .../hive/ql/exec/spark/TestSparkStatistics.java | 3 +++ .../jdbc/TestJdbcWithLocalClusterSpark.java | 3 +++ ...stMultiSessionsHS2WithLocalClusterSpark.java | 3 +++ .../session/TestSparkSessionManagerImpl.java| 24 .../hive/spark/client/TestSparkClient.java | 4 5 files changed, 27 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/068d007b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java index be3b501..4413161 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Assert; import org.junit.Test; +import java.nio.file.Paths; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -47,6 +48,8 @@ public class TestSparkStatistics { conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); conf.setVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "spark"); conf.set("spark.master", "local-cluster[1,2,1024]"); +conf.set("spark.local.dir", Paths.get(System.getProperty("test.tmp.dir"), +"TestSparkStatistics-local-dir").toString()); SessionState.start(conf); http://git-wip-us.apache.org/repos/asf/hive/blob/068d007b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java index 2a4da20..fe8a32f 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.nio.file.Paths; import java.sql.Connection; import java.sql.DriverManager; import java.sql.ResultSet; @@ -76,6 +77,8 @@ public class TestJdbcWithLocalClusterSpark { // Spark requires Hive to support Hadoop3 first then Spark can start // working on Hadoop3 support. Remove this after Spark supports Hadoop3. conf.set("dfs.client.datanode-restart.timeout", "30"); +conf.set("spark.local.dir", Paths.get(System.getProperty("test.tmp.dir"), +"TestJdbcWithLocalClusterSpark-local-dir").toString()); return conf; } http://git-wip-us.apache.org/repos/asf/hive/blob/068d007b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java index 9f72e51..79d56f5 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java @@ -18,6 +18,7 @@ package org.apache.hive.jdbc; +import java.nio.file.Paths; import java.sql.Connection; import java.sql.DriverManager; import java.sql.ResultSet; @@ -85,6 +86,8 @@ public class TestMultiSessionsHS2WithLocalClusterSpark { // Spark requires Hive to support Hadoop3 first then Spark can start // working on Hadoop3 support. Remove this after Spark supports Hadoop3. conf.set("dfs.client.datanode-restart.timeout", "30"); +
hive git commit: HIVE-18117: Create TestCliDriver for HDFS EC (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master eb21132bf -> 87e8c738b HIVE-18117: Create TestCliDriver for HDFS EC (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/87e8c738 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/87e8c738 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/87e8c738 Branch: refs/heads/master Commit: 87e8c738beab1ef2899681a13dec52acc6bd05f7 Parents: eb21132 Author: Andrew ShermanAuthored: Wed May 23 10:52:19 2018 -0500 Committer: Sahil Takiar Committed: Fri May 25 12:33:46 2018 -0700 -- .../cli/TestErasureCodingHDFSCliDriver.java | 65 .../test/resources/testconfiguration.properties | 6 + .../hadoop/hive/cli/control/CliConfigs.java | 50 +++ .../org/apache/hadoop/hive/ql/QTestUtil.java| 60 ++- .../ql/processors/CommandProcessorFactory.java | 65 ++-- .../hive/ql/processors/ErasureProcessor.java| 383 +++ .../hadoop/hive/ql/processors/HiveCommand.java | 1 + .../hadoop/hive/ql/session/SessionState.java| 5 + .../processors/TestCommandProcessorFactory.java | 2 +- .../queries/clientpositive/erasure_commands.q | 10 + .../queries/clientpositive/erasure_simple.q | 51 +++ .../clientpositive/erasure_commands.q.out | 4 + .../erasurecoding/erasure_commands.q.out| 8 + .../erasurecoding/erasure_simple.q.out | 111 ++ .../apache/hadoop/hive/shims/Hadoop23Shims.java | 164 .../apache/hadoop/hive/shims/HadoopShims.java | 107 ++ .../ptest2/conf/deployed/master-mr2.properties | 7 + 17 files changed, 1059 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/87e8c738/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java -- diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java new file mode 100644 index 000..b459ffc --- /dev/null +++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.cli; + +import java.io.File; +import java.util.List; + +import org.apache.hadoop.hive.cli.control.CliAdapter; +import org.apache.hadoop.hive.cli.control.CliConfigs; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +/** + * A Test Driver that can be used to run tests over hdfs directories that employ Erasure Coding. + */ +@RunWith(Parameterized.class) +public class TestErasureCodingHDFSCliDriver { + + static CliAdapter adapter = new CliConfigs.ErasureCodingHDFSCliConfig().getCliAdapter(); + + @Parameters(name = "{0}") + public static List
hive git commit: HIVE-19559: SparkClientImpl shouldn't name redirector thread RemoteDriver (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 92dd70739 -> 68a2beabc HIVE-19559: SparkClientImpl shouldn't name redirector thread RemoteDriver (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/68a2beab Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/68a2beab Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/68a2beab Branch: refs/heads/master Commit: 68a2beabc38944c3e5238b454b1a0fe30dbf0034 Parents: 92dd707 Author: Bharathkrishna Guruvayoor MuraliAuthored: Thu May 24 13:40:36 2018 -0700 Committer: Sahil Takiar Committed: Thu May 24 13:40:36 2018 -0700 -- .../main/java/org/apache/hive/spark/client/SparkClientImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/68a2beab/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java index f8b5d19..847c82b 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java @@ -473,9 +473,9 @@ class SparkClientImpl implements SparkClient { final List childErrorLog = Collections.synchronizedList(new ArrayList()); final LogRedirector.LogSourceCallback callback = () -> {return isAlive;}; -LogRedirector.redirect("RemoteDriver-stdout-redir-" + threadName, +LogRedirector.redirect("spark-submit-stdout-redir-" + threadName, new LogRedirector(child.getInputStream(), LOG, callback)); -LogRedirector.redirect("RemoteDriver-stderr-redir-" + threadName, +LogRedirector.redirect("spark-submit-stderr-redir-" + threadName, new LogRedirector(child.getErrorStream(), LOG, childErrorLog, callback)); runnable = new Runnable() {
hive git commit: HIVE-18117: Create TestCliDriver for HDFS EC (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 0b2d364aa -> c8889f86d HIVE-18117: Create TestCliDriver for HDFS EC (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c8889f86 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c8889f86 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c8889f86 Branch: refs/heads/master Commit: c8889f86d2f1e1f49064b05eff2c1b93504ba5bc Parents: 0b2d364 Author: Andrew ShermanAuthored: Wed May 23 10:52:19 2018 -0500 Committer: Sahil Takiar Committed: Wed May 23 10:52:19 2018 -0500 -- .../cli/TestErasureCodingHDFSCliDriver.java | 65 .../test/resources/testconfiguration.properties | 6 + .../hadoop/hive/cli/control/CliConfigs.java | 50 +++ .../org/apache/hadoop/hive/ql/QTestUtil.java| 60 ++- .../ql/processors/CommandProcessorFactory.java | 65 ++-- .../hive/ql/processors/ErasureProcessor.java| 383 +++ .../hadoop/hive/ql/processors/HiveCommand.java | 1 + .../hadoop/hive/ql/session/SessionState.java| 5 + .../processors/TestCommandProcessorFactory.java | 2 +- .../queries/clientpositive/erasure_commands.q | 10 + .../queries/clientpositive/erasure_simple.q | 51 +++ .../clientpositive/erasure_commands.q.out | 4 + .../erasurecoding/erasure_commands.q.out| 8 + .../erasurecoding/erasure_simple.q.out | 111 ++ .../apache/hadoop/hive/shims/Hadoop23Shims.java | 164 .../apache/hadoop/hive/shims/HadoopShims.java | 107 ++ .../ptest2/conf/deployed/master-mr2.properties | 7 + 17 files changed, 1059 insertions(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/c8889f86/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java -- diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java new file mode 100644 index 000..b459ffc --- /dev/null +++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestErasureCodingHDFSCliDriver.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.cli; + +import java.io.File; +import java.util.List; + +import org.apache.hadoop.hive.cli.control.CliAdapter; +import org.apache.hadoop.hive.cli.control.CliConfigs; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +/** + * A Test Driver that can be used to run tests over hdfs directories that employ Erasure Coding. + */ +@RunWith(Parameterized.class) +public class TestErasureCodingHDFSCliDriver { + + static CliAdapter adapter = new CliConfigs.ErasureCodingHDFSCliConfig().getCliAdapter(); + + @Parameters(name = "{0}") + public static List
hive git commit: HIVE-19421: Upgrade version of Jetty to 9.3.20.v20170531 (Janaki Lahorani, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 38f7a7f38 -> 610748287 HIVE-19421: Upgrade version of Jetty to 9.3.20.v20170531 (Janaki Lahorani, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/61074828 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/61074828 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/61074828 Branch: refs/heads/master Commit: 610748287846cbd26d0b7c8ccc414f8636fb6ba1 Parents: 38f7a7f Author: Janaki LahoraniAuthored: Mon May 21 15:29:45 2018 -0500 Committer: Sahil Takiar Committed: Mon May 21 15:30:11 2018 -0500 -- .../src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/61074828/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java index f45fb50..f52338a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java @@ -976,7 +976,7 @@ public class TestJdbcWithMiniHS2 { } } -// This should fail with given HTTP response code 413 in error message, since header is more +// This should fail with given HTTP response code 431 in error message, since header is more // than the configured the header size password = StringUtils.leftPad("*", 2000); Exception headerException = null; @@ -992,7 +992,7 @@ public class TestJdbcWithMiniHS2 { assertTrue("Header exception should be thrown", headerException != null); assertTrue("Incorrect HTTP Response:" + headerException.getMessage(), - headerException.getMessage().contains("HTTP Response code: 413")); + headerException.getMessage().contains("HTTP Response code: 431")); } // Stop HiveServer2 to increase header size http://git-wip-us.apache.org/repos/asf/hive/blob/61074828/pom.xml -- diff --git a/pom.xml b/pom.xml index ce3da37..df10c74 100644 --- a/pom.xml +++ b/pom.xml @@ -168,7 +168,7 @@ 5.5.1 3.0.1 1.1 -9.3.8.v20160314 +9.3.20.v20170531 1.19 2.22.2
hive git commit: HIVE-19562: Flaky test: TestMiniSparkOnYarn FileNotFoundException in spark-submit (Sahil Takiar, reviewed by Peter Vary) (addendum)
Repository: hive Updated Branches: refs/heads/master 6eaef86ea -> 0b7cdde00 HIVE-19562: Flaky test: TestMiniSparkOnYarn FileNotFoundException in spark-submit (Sahil Takiar, reviewed by Peter Vary) (addendum) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0b7cdde0 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0b7cdde0 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0b7cdde0 Branch: refs/heads/master Commit: 0b7cdde00e3703858fb0d6d89b08d9ede309e0b3 Parents: 6eaef86 Author: Sahil TakiarAuthored: Fri May 18 17:00:59 2018 -0500 Committer: Sahil Takiar Committed: Fri May 18 17:00:59 2018 -0500 -- data/conf/perf-reg/spark/hive-site.xml | 5 + 1 file changed, 5 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/0b7cdde0/data/conf/perf-reg/spark/hive-site.xml -- diff --git a/data/conf/perf-reg/spark/hive-site.xml b/data/conf/perf-reg/spark/hive-site.xml index 8a32afe..e52a2f9 100644 --- a/data/conf/perf-reg/spark/hive-site.xml +++ b/data/conf/perf-reg/spark/hive-site.xml @@ -245,6 +245,11 @@ + spark.local.dir + ${test.tmp.dir}/spark-local-dir + + + hive.aux.jars.path ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar
hive git commit: HIVE-19562: Flaky test: TestMiniSparkOnYarn FileNotFoundException in spark-submit (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 0b6bea89f -> 0420bde92 HIVE-19562: Flaky test: TestMiniSparkOnYarn FileNotFoundException in spark-submit (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0420bde9 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0420bde9 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0420bde9 Branch: refs/heads/master Commit: 0420bde9275fccec4bfa9ee927713e7b68c88634 Parents: 0b6bea8 Author: Sahil TakiarAuthored: Fri May 18 08:22:28 2018 -0500 Committer: Sahil Takiar Committed: Fri May 18 08:22:28 2018 -0500 -- data/conf/spark/local/hive-site.xml| 5 + data/conf/spark/standalone/hive-site.xml | 5 + data/conf/spark/yarn-cluster/hive-site.xml | 5 + 3 files changed, 15 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/0420bde9/data/conf/spark/local/hive-site.xml -- diff --git a/data/conf/spark/local/hive-site.xml b/data/conf/spark/local/hive-site.xml index 8ff6256..c26bfaa 100644 --- a/data/conf/spark/local/hive-site.xml +++ b/data/conf/spark/local/hive-site.xml @@ -240,6 +240,11 @@ + spark.local.dir + ${test.tmp.dir}/spark-local-dir + + + hive.aux.jars.path ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar http://git-wip-us.apache.org/repos/asf/hive/blob/0420bde9/data/conf/spark/standalone/hive-site.xml -- diff --git a/data/conf/spark/standalone/hive-site.xml b/data/conf/spark/standalone/hive-site.xml index 84851c7..79e388e 100644 --- a/data/conf/spark/standalone/hive-site.xml +++ b/data/conf/spark/standalone/hive-site.xml @@ -240,6 +240,11 @@ + spark.local.dir + ${test.tmp.dir}/spark-local-dir + + + hive.aux.jars.path ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar http://git-wip-us.apache.org/repos/asf/hive/blob/0420bde9/data/conf/spark/yarn-cluster/hive-site.xml -- diff --git a/data/conf/spark/yarn-cluster/hive-site.xml b/data/conf/spark/yarn-cluster/hive-site.xml index 265e398..94af267 100644 --- a/data/conf/spark/yarn-cluster/hive-site.xml +++ b/data/conf/spark/yarn-cluster/hive-site.xml @@ -260,6 +260,11 @@ + spark.local.dir + ${test.tmp.dir}/spark-local-dir + + + hive.prewarm.enabled true
hive git commit: HIVE-17838: Make org.apache.hive.spark.client.rpc logging HoS specific and other logging cleanup (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 8158f8848 -> 52f1b2471 HIVE-17838: Make org.apache.hive.spark.client.rpc logging HoS specific and other logging cleanup (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/52f1b247 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/52f1b247 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/52f1b247 Branch: refs/heads/master Commit: 52f1b2471545a797856e4b9b1ae0a36cb4233c18 Parents: 8158f88 Author: Sahil TakiarAuthored: Fri May 4 14:32:50 2018 -0700 Committer: Sahil Takiar Committed: Fri May 4 14:32:50 2018 -0700 -- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 8 +-- .../ql/exec/spark/HiveSparkClientFactory.java | 4 +- .../hadoop/hive/ql/exec/spark/SparkTask.java| 6 +-- .../ql/exec/spark/session/SparkSessionImpl.java | 28 ++- .../spark/SetSparkReducerParallelism.java | 4 +- .../apache/hive/spark/client/BaseProtocol.java | 53 .../apache/hive/spark/client/RemoteDriver.java | 39 -- .../hive/spark/client/SparkClientFactory.java | 4 +- .../hive/spark/client/SparkClientImpl.java | 44 ++-- .../hive/spark/client/metrics/InputMetrics.java | 6 +++ .../hive/spark/client/metrics/Metrics.java | 18 +++ .../client/metrics/ShuffleReadMetrics.java | 9 .../client/metrics/ShuffleWriteMetrics.java | 7 +++ .../hive/spark/client/rpc/KryoMessageCodec.java | 4 +- .../org/apache/hive/spark/client/rpc/Rpc.java | 23 ++--- .../hive/spark/client/rpc/RpcConfiguration.java | 9 ++-- .../hive/spark/client/rpc/RpcDispatcher.java| 20 .../apache/hive/spark/client/rpc/RpcServer.java | 18 --- 18 files changed, 216 insertions(+), 88 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/52f1b247/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 99df967..8baf309 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -495,8 +495,8 @@ public enum ErrorMsg { FILE_NOT_FOUND(20012, "File not found: {0}", "64000", true), WRONG_FILE_FORMAT(20013, "Wrong file format. Please check the file's format.", "64000", true), - SPARK_CREATE_CLIENT_INVALID_QUEUE(20014, "Spark job is submitted to an invalid queue: {0}." - + " Please fix and try again.", true), + SPARK_CREATE_CLIENT_INVALID_QUEUE(20014, "Spark app for session {0} was submitted to an invalid" + + " queue: {1}. Please fix and try again.", true), SPARK_RUNTIME_OOM(20015, "Spark job failed because of out of memory."), // An exception from runtime that will show the full stack to client @@ -574,13 +574,13 @@ public enum ErrorMsg { SPARK_CREATE_CLIENT_TIMEOUT(30038, "Timed out while creating Spark client for session {0}.", true), SPARK_CREATE_CLIENT_QUEUE_FULL(30039, - "Failed to create Spark client because job queue is full: {0}.", true), + "Failed to create Spark client for session {0} because job queue is full: {1}.", true), SPARK_CREATE_CLIENT_INTERRUPTED(30040, "Interrupted while creating Spark client for session {0}", true), SPARK_CREATE_CLIENT_ERROR(30041, "Failed to create Spark client for Spark session {0}: {1}", true), SPARK_CREATE_CLIENT_INVALID_RESOURCE_REQUEST(30042, - "Failed to create Spark client due to invalid resource request: {0}", true), + "Failed to create Spark client for session {0} due to invalid resource request: {1}", true), SPARK_CREATE_CLIENT_CLOSED_SESSION(30043, "Cannot create Spark client on a closed session {0}", true), http://git-wip-us.apache.org/repos/asf/hive/blob/52f1b247/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index 565c43b..5ed5d42 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -104,7 +104,7 @@ public class HiveSparkClientFactory { inputStream = HiveSparkClientFactory.class.getClassLoader() .getResourceAsStream(SPARK_DEFAULT_CONF_FILE); if (inputStream != null) { -LOG.info("loading spark properties from: " + SPARK_DEFAULT_CONF_FILE); +
hive git commit: HIVE-19212: Fix findbugs yetus pre-commit checks (Sahil Takiar, reviewed by Adam Szita, Peter Vary)
Repository: hive Updated Branches: refs/heads/master 70d835b98 -> bf8e69643 HIVE-19212: Fix findbugs yetus pre-commit checks (Sahil Takiar, reviewed by Adam Szita, Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bf8e6964 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bf8e6964 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bf8e6964 Branch: refs/heads/master Commit: bf8e6964389c984d768f6a4b110f2ba8198a49ef Parents: 70d835b Author: Sahil TakiarAuthored: Thu May 3 15:31:09 2018 -0700 Committer: Sahil Takiar Committed: Thu May 3 15:41:41 2018 -0700 -- dev-support/yetus-wrapper.sh| 47 +--- .../apache/hive/ptest/execution/YetusPhase.java | 9 ++-- .../ptest2/src/main/resources/yetus-exec.vm | 26 +++ 3 files changed, 41 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/bf8e6964/dev-support/yetus-wrapper.sh -- diff --git a/dev-support/yetus-wrapper.sh b/dev-support/yetus-wrapper.sh index 58da1d2..3a814d5 100755 --- a/dev-support/yetus-wrapper.sh +++ b/dev-support/yetus-wrapper.sh @@ -91,34 +91,6 @@ if [[ $? != 0 ]]; then fi HIVE_PATCHPROCESS=${mytmpdir} -CURLBIN=$(command -v curl) - -# Set FindBugs Home -FINDBUGS_VERSION="3.0.1" -if [[ ! -d "${HIVE_PATCHPROCESS}/findbugs-${FINDBUGS_VERSION}/" ]]; then - # Download FindBugs - FINDBUGS_BASEURL="http://prdownloads.sourceforge.net/findbugs/; - FINDBUGS_TARBALL="findbugs-${FINDBUGS_VERSION}.tar" - - pushd "${HIVE_PATCHPROCESS}" >/dev/null - if [[ -n "${CURLBIN}" ]]; then -"${CURLBIN}" -f -s -L -O "${FINDBUGS_BASEURL}/${FINDBUGS_TARBALL}.gz" -if [[ $? != 0 ]]; then - yetus_error "ERROR: yetus-dl: unable to download ${FINDBUGS_BASEURL}/${FINDBUGS_TARBALL}.gz" - exit 1 -fi - fi - - gunzip -c "${FINDBUGS_TARBALL}.gz" | tar xpf - - if [[ $? != 0 ]]; then -yetus_error "ERROR: ${FINDBUGS_TARBALL}.gz is corrupt. Investigate and then remove ${HIVE_PATCHPROCESS} to try again." -exit 1 - fi - popd >/dev/null -fi - -export FINDBUGS_HOME=${HIVE_PATCHPROCESS}/findbugs-${FINDBUGS_VERSION} - ## ## if we've already DL'd it, then short cut ## @@ -130,10 +102,11 @@ fi ## need to DL, etc ## -YETUS_BASEURL="https://archive.apache.org/dist/yetus/${HIVE_YETUS_VERSION}/; -YETUS_TARBALL="yetus-${HIVE_YETUS_VERSION}-bin.tar" +BASEURL="https://archive.apache.org/dist/yetus/${HIVE_YETUS_VERSION}/; +TARBALL="yetus-${HIVE_YETUS_VERSION}-bin.tar" GPGBIN=$(command -v gpg) +CURLBIN=$(command -v curl) pushd "${HIVE_PATCHPROCESS}" >/dev/null if [[ $? != 0 ]]; then @@ -142,9 +115,9 @@ if [[ $? != 0 ]]; then fi if [[ -n "${CURLBIN}" ]]; then - "${CURLBIN}" -f -s -L -O "${YETUS_BASEURL}/${YETUS_TARBALL}.gz" + "${CURLBIN}" -f -s -L -O "${BASEURL}/${TARBALL}.gz" if [[ $? != 0 ]]; then -yetus_error "ERROR: yetus-dl: unable to download ${YETUS_BASEURL}/${YETUS_TARBALL}.gz" +yetus_error "ERROR: yetus-dl: unable to download ${BASEURL}/${TARBALL}.gz" exit 1 fi else @@ -168,9 +141,9 @@ if [[ -n "${GPGBIN}" ]]; then yetus_error "ERROR: yetus-dl: unable to fetch https://dist.apache.org/repos/dist/release/yetus/KEYS; exit 1 fi - "${CURLBIN}" -s -L -O "${YETUS_BASEURL}/${YETUS_TARBALL}.gz.asc" + "${CURLBIN}" -s -L -O "${BASEURL}/${TARBALL}.gz.asc" if [[ $? != 0 ]]; then -yetus_error "ERROR: yetus-dl: unable to fetch ${YETUS_BASEURL}/${YETUS_TARBALL}.gz.asc" +yetus_error "ERROR: yetus-dl: unable to fetch ${BASEURL}/${TARBALL}.gz.asc" exit 1 fi "${GPGBIN}" --homedir "${HIVE_PATCHPROCESS}/.gpg" --import "${HIVE_PATCHPROCESS}/KEYS_YETUS" >/dev/null 2>&1 @@ -178,16 +151,16 @@ if [[ -n "${GPGBIN}" ]]; then yetus_error "ERROR: yetus-dl: gpg unable to import ${HIVE_PATCHPROCESS}/KEYS_YETUS" exit 1 fi - "${GPGBIN}" --homedir "${HIVE_PATCHPROCESS}/.gpg" --verify "${YETUS_TARBALL}.gz.asc" >/dev/null 2>&1 + "${GPGBIN}" --homedir "${HIVE_PATCHPROCESS}/.gpg" --verify "${TARBALL}.gz.asc" >/dev/null 2>&1 if [[ $? != 0 ]]; then yetus_error "ERROR: yetus-dl: gpg verify of tarball in ${HIVE_PATCHPROCESS} failed" exit 1 fi fi -gunzip -c "${YETUS_TARBALL}.gz" | tar xpf - +gunzip -c "${TARBALL}.gz" | tar xpf - if [[ $? != 0 ]]; then - yetus_error "ERROR: ${YETUS_TARBALL}.gz is corrupt. Investigate and then remove ${HIVE_PATCHPROCESS} to try again." + yetus_error "ERROR: ${TARBALL}.gz is corrupt. Investigate and then remove ${HIVE_PATCHPROCESS} to try again." exit 1 fi http://git-wip-us.apache.org/repos/asf/hive/blob/bf8e6964/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/YetusPhase.java
hive git commit: HIVE-18958: Fix Spark config warnings (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 758b913f2 -> 41de95318 HIVE-18958: Fix Spark config warnings (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/41de9531 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/41de9531 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/41de9531 Branch: refs/heads/master Commit: 41de95318d80df282fbed17ede6b3a05f649cce9 Parents: 758b913 Author: Bharathkrishna Guruvayoor MuraliAuthored: Tue May 1 09:15:29 2018 -0500 Committer: Sahil Takiar Committed: Tue May 1 09:15:29 2018 -0500 -- data/conf/spark/yarn-client/hive-site.xml | 314 --- data/conf/spark/yarn-cluster/hive-site.xml | 314 +++ .../hadoop/hive/cli/control/CliConfigs.java | 2 +- .../ql/exec/spark/HiveSparkClientFactory.java | 8 - .../apache/hive/spark/client/RemoteDriver.java | 23 +- .../hive/spark/client/SparkClientImpl.java | 9 +- 6 files changed, 332 insertions(+), 338 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/41de9531/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 deleted file mode 100644 index 6c63362..000 --- a/data/conf/spark/yarn-client/hive-site.xml +++ /dev/null @@ -1,314 +0,0 @@ - - - - - - - - - - - - - - - hadoop.tmp.dir - ${test.tmp.dir}/hadoop-tmp - A base for other temporary directories. - - - - hive.exec.scratchdir - ${test.tmp.dir}/scratchdir - Scratch space for Hive jobs - - - - hive.exec.local.scratchdir - ${test.tmp.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - datanucleus.schema.autoCreateAll - true - - - - hive.metastore.schema.verification - false - - - - javax.jdo.option.ConnectionURL - jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - file://${test.tmp.dir}/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${test.tmp.dir}/log/ - - - - - test.data.files - ${hive.root}/data/files - - - - - hive.jar.path - ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - - test.data.scripts - ${hive.root}/data/scripts - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${test.tmp.dir}/tmp - Location of the structured hive logs - - - - hive.exec.pre.hooks - org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables - Pre Execute Hook for Tests - - - - hive.exec.post.hooks - org.apache.hadoop.hive.ql.hooks.PostExecutePrinter - Post Execute Hook for Tests - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - -Let hive determine whether to run in local mode automatically -Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - false - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - false - Whether Hive ignores the mapjoin hint - - - - io.sort.mb - 10 - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - - hive.stats.dbclass - fs - The default storatge
hive git commit: HIVE-18903: Lower Logging Level for ObjectStore (Antal Sinkovits, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 34298eb81 -> 331dd57a0 HIVE-18903: Lower Logging Level for ObjectStore (Antal Sinkovits, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/331dd57a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/331dd57a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/331dd57a Branch: refs/heads/master Commit: 331dd57a0f9f8263d4e4a972042735d577d1240e Parents: 34298eb Author: Antal SinkovitsAuthored: Fri Apr 27 13:02:20 2018 -0500 Committer: Sahil Takiar Committed: Fri Apr 27 13:02:20 2018 -0500 -- .../main/java/org/apache/hadoop/hive/metastore/ObjectStore.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/331dd57a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java -- diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 1abd99d..2bd7c21 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -392,7 +392,7 @@ public class ObjectStore implements RawStore, Configurable { throw new RuntimeException( "Unable to create persistence manager. Check dss.log for details"); } else { -LOG.info("Initialized ObjectStore"); +LOG.debug("Initialized ObjectStore"); } } finally { pmfPropLock.unlock(); @@ -477,7 +477,7 @@ public class ObjectStore implements RawStore, Configurable { * @param dsProps */ private void initializeHelper(Properties dsProps) { -LOG.info("ObjectStore, initialize called"); +LOG.debug("ObjectStore, initialize called"); prop = dsProps; pm = getPersistenceManager(); try {
hive git commit: HIVE-18880: Change Log to Debug in CombineHiveInputFormat (Antal Sinkovits, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master e0b318269 -> 34298eb81 HIVE-18880: Change Log to Debug in CombineHiveInputFormat (Antal Sinkovits, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/34298eb8 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/34298eb8 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/34298eb8 Branch: refs/heads/master Commit: 34298eb81d20a4a5bafd52fcf2f5dd45722aac05 Parents: e0b3182 Author: Antal SinkovitsAuthored: Fri Apr 27 12:56:00 2018 -0500 Committer: Sahil Takiar Committed: Fri Apr 27 12:56:00 2018 -0500 -- .../java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/34298eb8/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java index 1622ae2..5f2539f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java @@ -464,7 +464,7 @@ public class CombineHiveInputFormat
hive git commit: HIVE-16861: MapredParquetOutputFormat - Save Some Array Allocations (BELUGA BEHR, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 23d20e649 -> 823aadad5 HIVE-16861: MapredParquetOutputFormat - Save Some Array Allocations (BELUGA BEHR, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/823aadad Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/823aadad Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/823aadad Branch: refs/heads/master Commit: 823aadad5fbfb51a2eb4c07d8cb0044156699126 Parents: 23d20e6 Author: BELUGA BEHRAuthored: Thu Apr 26 12:06:43 2018 -0500 Committer: Sahil Takiar Committed: Thu Apr 26 12:07:11 2018 -0500 -- .../ql/io/parquet/MapredParquetOutputFormat.java | 17 +++-- 1 file changed, 7 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/823aadad/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java index 6e05526..ba235f7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetOutputFormat.java @@ -14,8 +14,8 @@ package org.apache.hadoop.hive.ql.io.parquet; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Properties; @@ -101,23 +101,20 @@ public class MapredParquetOutputFormat extends FileOutputFormat
hive git commit: HIVE-19158: Fix NPE in the HiveMetastore add partition tests (Marta Kuczora, reviewed by Peter Vary and Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master fb22f576d -> 1f25c46a2 HIVE-19158: Fix NPE in the HiveMetastore add partition tests (Marta Kuczora, reviewed by Peter Vary and Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1f25c46a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1f25c46a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1f25c46a Branch: refs/heads/master Commit: 1f25c46a2bf50483e09c756803d78e078dc37b92 Parents: fb22f57 Author: Marta KuczoraAuthored: Thu Apr 19 11:06:45 2018 -0500 Committer: Sahil Takiar Committed: Thu Apr 19 11:06:45 2018 -0500 -- .../hadoop/hive/metastore/HiveMetaStore.java| 26 +- .../hive/metastore/HiveMetaStoreClient.java | 11 ++- .../spec/CompositePartitionSpecProxy.java | 6 +- .../spec/PartitionListComposingSpecProxy.java | 22 - .../partition/spec/PartitionSpecProxy.java | 6 +- .../spec/PartitionSpecWithSharedSDProxy.java| 5 +- .../metastore/client/TestAddPartitions.java | 71 ++-- .../client/TestAddPartitionsFromPartSpec.java | 89 ++-- 8 files changed, 116 insertions(+), 120 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java -- diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index ae9ec5c..9c88cf9 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -3250,6 +3250,12 @@ public class HiveMetaStore extends ThriftHiveMetastore { part.getTableName(), part.toString()); throw new MetaException(errorMsg); } + if (part.getValues() == null || part.getValues().isEmpty()) { +throw new MetaException("The partition values cannot be null or empty."); + } + if (part.getValues().contains(null)) { +throw new MetaException("Partition value cannot be null."); + } boolean shouldAdd = startAddPartition(ms, part, ifNotExists); if (!shouldAdd) { @@ -3410,7 +3416,10 @@ public class HiveMetaStore extends ThriftHiveMetastore { public int add_partitions(final List parts) throws MetaException, InvalidObjectException, AlreadyExistsException { startFunction("add_partition"); - if (parts.size() == 0) { + if (parts == null) { +throw new MetaException("Partition list cannot be null."); + } + if (parts.isEmpty()) { return 0; } @@ -3471,6 +3480,9 @@ public class HiveMetaStore extends ThriftHiveMetastore { boolean ifNotExists) throws TException { boolean success = false; + if (dbName == null || tblName == null) { +throw new MetaException("The database and table name cannot be null."); + } // Ensures that the list doesn't have dups, and keeps track of directories we have created. final Map addedPartitions = new ConcurrentHashMap<>(); PartitionSpecProxy partitionSpecProxy = PartitionSpecProxy.Factory.get(partSpecs); @@ -3496,12 +3508,18 @@ public class HiveMetaStore extends ThriftHiveMetastore { // will be created if the list contains an invalid partition. final Partition part = partitionIterator.getCurrent(); + if (part.getDbName() == null || part.getTableName() == null) { +throw new MetaException("The database and table name must be set in the partition."); + } if (!part.getTableName().equalsIgnoreCase(tblName) || !part.getDbName().equalsIgnoreCase(dbName)) { String errorMsg = String.format( "Partition does not belong to target table %s.%s. It belongs to the table %s.%s : %s", dbName, tblName, part.getDbName(), part.getTableName(), part.toString()); throw new MetaException(errorMsg); } + if (part.getValues() == null || part.getValues().isEmpty()) { +throw new MetaException("The partition values cannot be null or empty."); + } boolean shouldAdd = startAddPartition(ms, part, ifNotExists); if (!shouldAdd) { @@ -3733,6 +3751,9 @@ public class HiveMetaStore extends ThriftHiveMetastore { firePreEvent(new PreAddPartitionEvent(tbl, part, this)); +if
hive git commit: HIVE-18319: Upgrade to Hadoop 3.0.0 (Sahil Takiar, reviewed by Ashutosh Chauhan)
Repository: hive Updated Branches: refs/heads/master 2f0b41bc6 -> 16d94fbe2 HIVE-18319: Upgrade to Hadoop 3.0.0 (Sahil Takiar, 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/16d94fbe Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/16d94fbe Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/16d94fbe Branch: refs/heads/master Commit: 16d94fbe21de4a817d640ae05f7b324d207d42e1 Parents: 2f0b41b Author: Sahil TakiarAuthored: Sun Apr 15 20:40:51 2018 -0700 Committer: Sahil Takiar Committed: Sun Apr 15 20:40:51 2018 -0700 -- pom.xml| 2 +- .../common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/16d94fbe/pom.xml -- diff --git a/pom.xml b/pom.xml index ca915ef..2d30789 100644 --- a/pom.xml +++ b/pom.xml @@ -144,7 +144,7 @@ 19.0 2.4.11 1.3.166 -3.0.0-beta1 +3.0.0 ${basedir}/${hive.path.to.root}/testutils/hadoop 1.3 2.0.0-alpha4 http://git-wip-us.apache.org/repos/asf/hive/blob/16d94fbe/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java -- diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java index a28b595..facad8a 100644 --- a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java +++ b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java @@ -38,7 +38,6 @@ import org.apache.hadoop.fs.permission.AclEntryType; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory;
hive git commit: HIVE-18525: Add explain plan to Hive on Spark Web UI (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 3915980fe -> b14113be4 HIVE-18525: Add explain plan to Hive on Spark Web UI (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b14113be Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b14113be Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b14113be Branch: refs/heads/master Commit: b14113be4c67217e030ae5158ce584362a721483 Parents: 3915980 Author: Sahil TakiarAuthored: Fri Apr 13 13:50:57 2018 -0700 Committer: Sahil Takiar Committed: Fri Apr 13 13:50:57 2018 -0700 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 6 +- .../apache/hadoop/hive/ql/log/PerfLogger.java | 1 + .../apache/hadoop/hive/ql/exec/ExplainTask.java | 22 ++-- .../hadoop/hive/ql/exec/spark/CacheTran.java| 10 +- .../hadoop/hive/ql/exec/spark/MapInput.java | 13 +- .../hadoop/hive/ql/exec/spark/MapTran.java | 7 +- .../hadoop/hive/ql/exec/spark/ReduceTran.java | 7 +- .../hadoop/hive/ql/exec/spark/ShuffleTran.java | 12 +- .../hadoop/hive/ql/exec/spark/SparkPlan.java| 57 - .../hive/ql/exec/spark/SparkPlanGenerator.java | 14 +-- .../hadoop/hive/ql/exec/spark/SparkTran.java| 3 + .../hive/ql/exec/spark/TestSparkPlan.java | 122 +++ 12 files changed, 243 insertions(+), 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b14113be/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 e540d02..e533ee6 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2830,8 +2830,12 @@ public class HiveConf extends Configuration { HIVE_SPARK_EXPLAIN_USER("hive.spark.explain.user", false, "Whether to show explain result at user level.\n" + "When enabled, will log EXPLAIN output for the query at user level. Spark only."), +HIVE_SPARK_LOG_EXPLAIN_WEBUI("hive.spark.log.explain.webui", true, "Whether to show the " + +"explain plan in the Spark Web UI. Only shows the regular EXPLAIN plan, and ignores " + +"any extra EXPLAIN configuration (e.g. hive.spark.explain.user, etc.). The explain " + +"plan for each stage is truncated at 100,000 characters."), -// prefix used to auto generated column aliases (this should be started with '_') +// prefix used to auto generated column aliases (this should be s,tarted with '_') HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL("hive.autogen.columnalias.prefix.label", "_c", "String used as a prefix when auto generating column alias.\n" + "By default the prefix label will be appended with a column position number to form the column alias. \n" + http://git-wip-us.apache.org/repos/asf/hive/blob/b14113be/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java -- diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java index 764a832..c1e1b7f 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java +++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java @@ -76,6 +76,7 @@ public class PerfLogger { public static final String SPARK_SUBMIT_TO_RUNNING = "SparkSubmitToRunning"; public static final String SPARK_BUILD_PLAN = "SparkBuildPlan"; public static final String SPARK_BUILD_RDD_GRAPH = "SparkBuildRDDGraph"; + public static final String SPARK_CREATE_EXPLAIN_PLAN = "SparkCreateExplainPlan."; public static final String SPARK_SUBMIT_JOB = "SparkSubmitJob"; public static final String SPARK_RUN_JOB = "SparkRunJob"; public static final String SPARK_CREATE_TRAN = "SparkCreateTran."; http://git-wip-us.apache.org/repos/asf/hive/blob/b14113be/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index 14c6398..0b30721 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -610,7 +610,7 @@ public class ExplainTask extends Task implements Serializable { } private JSONArray outputList(List l, PrintStream out, boolean hasHeader, - boolean extended, boolean jsonOutput, int indent) throws Exception { +
hive git commit: HIVE-18883: Add findbugs to yetus pre-commit checks (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 8777125b3 -> 42187fdbc HIVE-18883: Add findbugs to yetus pre-commit checks (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/42187fdb Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/42187fdb Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/42187fdb Branch: refs/heads/master Commit: 42187fdbc3de0be793319483e2f463b3e8756f20 Parents: 8777125 Author: Sahil TakiarAuthored: Wed Apr 11 11:01:11 2018 -0700 Committer: Sahil Takiar Committed: Wed Apr 11 12:08:24 2018 -0700 -- dev-support/yetus-wrapper.sh| 47 .../findbugs/findbugs-exclude.xml | 24 + standalone-metastore/pom.xml| 57 +++- 3 files changed, 117 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/42187fdb/dev-support/yetus-wrapper.sh -- diff --git a/dev-support/yetus-wrapper.sh b/dev-support/yetus-wrapper.sh index 3a814d5..58da1d2 100755 --- a/dev-support/yetus-wrapper.sh +++ b/dev-support/yetus-wrapper.sh @@ -91,6 +91,34 @@ if [[ $? != 0 ]]; then fi HIVE_PATCHPROCESS=${mytmpdir} +CURLBIN=$(command -v curl) + +# Set FindBugs Home +FINDBUGS_VERSION="3.0.1" +if [[ ! -d "${HIVE_PATCHPROCESS}/findbugs-${FINDBUGS_VERSION}/" ]]; then + # Download FindBugs + FINDBUGS_BASEURL="http://prdownloads.sourceforge.net/findbugs/; + FINDBUGS_TARBALL="findbugs-${FINDBUGS_VERSION}.tar" + + pushd "${HIVE_PATCHPROCESS}" >/dev/null + if [[ -n "${CURLBIN}" ]]; then +"${CURLBIN}" -f -s -L -O "${FINDBUGS_BASEURL}/${FINDBUGS_TARBALL}.gz" +if [[ $? != 0 ]]; then + yetus_error "ERROR: yetus-dl: unable to download ${FINDBUGS_BASEURL}/${FINDBUGS_TARBALL}.gz" + exit 1 +fi + fi + + gunzip -c "${FINDBUGS_TARBALL}.gz" | tar xpf - + if [[ $? != 0 ]]; then +yetus_error "ERROR: ${FINDBUGS_TARBALL}.gz is corrupt. Investigate and then remove ${HIVE_PATCHPROCESS} to try again." +exit 1 + fi + popd >/dev/null +fi + +export FINDBUGS_HOME=${HIVE_PATCHPROCESS}/findbugs-${FINDBUGS_VERSION} + ## ## if we've already DL'd it, then short cut ## @@ -102,11 +130,10 @@ fi ## need to DL, etc ## -BASEURL="https://archive.apache.org/dist/yetus/${HIVE_YETUS_VERSION}/; -TARBALL="yetus-${HIVE_YETUS_VERSION}-bin.tar" +YETUS_BASEURL="https://archive.apache.org/dist/yetus/${HIVE_YETUS_VERSION}/; +YETUS_TARBALL="yetus-${HIVE_YETUS_VERSION}-bin.tar" GPGBIN=$(command -v gpg) -CURLBIN=$(command -v curl) pushd "${HIVE_PATCHPROCESS}" >/dev/null if [[ $? != 0 ]]; then @@ -115,9 +142,9 @@ if [[ $? != 0 ]]; then fi if [[ -n "${CURLBIN}" ]]; then - "${CURLBIN}" -f -s -L -O "${BASEURL}/${TARBALL}.gz" + "${CURLBIN}" -f -s -L -O "${YETUS_BASEURL}/${YETUS_TARBALL}.gz" if [[ $? != 0 ]]; then -yetus_error "ERROR: yetus-dl: unable to download ${BASEURL}/${TARBALL}.gz" +yetus_error "ERROR: yetus-dl: unable to download ${YETUS_BASEURL}/${YETUS_TARBALL}.gz" exit 1 fi else @@ -141,9 +168,9 @@ if [[ -n "${GPGBIN}" ]]; then yetus_error "ERROR: yetus-dl: unable to fetch https://dist.apache.org/repos/dist/release/yetus/KEYS; exit 1 fi - "${CURLBIN}" -s -L -O "${BASEURL}/${TARBALL}.gz.asc" + "${CURLBIN}" -s -L -O "${YETUS_BASEURL}/${YETUS_TARBALL}.gz.asc" if [[ $? != 0 ]]; then -yetus_error "ERROR: yetus-dl: unable to fetch ${BASEURL}/${TARBALL}.gz.asc" +yetus_error "ERROR: yetus-dl: unable to fetch ${YETUS_BASEURL}/${YETUS_TARBALL}.gz.asc" exit 1 fi "${GPGBIN}" --homedir "${HIVE_PATCHPROCESS}/.gpg" --import "${HIVE_PATCHPROCESS}/KEYS_YETUS" >/dev/null 2>&1 @@ -151,16 +178,16 @@ if [[ -n "${GPGBIN}" ]]; then yetus_error "ERROR: yetus-dl: gpg unable to import ${HIVE_PATCHPROCESS}/KEYS_YETUS" exit 1 fi - "${GPGBIN}" --homedir "${HIVE_PATCHPROCESS}/.gpg" --verify "${TARBALL}.gz.asc" >/dev/null 2>&1 + "${GPGBIN}" --homedir "${HIVE_PATCHPROCESS}/.gpg" --verify "${YETUS_TARBALL}.gz.asc" >/dev/null 2>&1 if [[ $? != 0 ]]; then yetus_error "ERROR: yetus-dl: gpg verify of tarball in ${HIVE_PATCHPROCESS} failed" exit 1 fi fi -gunzip -c "${TARBALL}.gz" | tar xpf - +gunzip -c "${YETUS_TARBALL}.gz" | tar xpf - if [[ $? != 0 ]]; then - yetus_error "ERROR: ${TARBALL}.gz is corrupt. Investigate and then remove ${HIVE_PATCHPROCESS} to try again." + yetus_error "ERROR: ${YETUS_TARBALL}.gz is corrupt. Investigate and then remove ${HIVE_PATCHPROCESS} to try again." exit 1 fi http://git-wip-us.apache.org/repos/asf/hive/blob/42187fdb/standalone-metastore/findbugs/findbugs-exclude.xml
hive git commit: HIVE-18651: Expose additional Spark metrics (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master 2f802e908 -> eb40ea57e HIVE-18651: Expose additional Spark metrics (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/eb40ea57 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/eb40ea57 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/eb40ea57 Branch: refs/heads/master Commit: eb40ea57eac4c3ff46f638cf4ab83bec71b5eda5 Parents: 2f802e9 Author: Sahil TakiarAuthored: Fri Apr 6 11:42:23 2018 +0700 Committer: Sahil Takiar Committed: Fri Apr 6 11:42:23 2018 +0700 -- .../hive/ql/exec/spark/TestSparkStatistics.java | 100 +++ .../hadoop/hive/ql/exec/spark/SparkTask.java| 18 +++- .../spark/Statistic/SparkStatisticsNames.java | 4 +- .../spark/status/impl/SparkMetricsUtils.java| 5 +- .../hive/ql/exec/spark/TestSparkTask.java | 14 +++ .../hive/spark/client/MetricsCollection.java| 6 ++ .../hive/spark/client/metrics/Metrics.java | 13 ++- .../spark/client/TestMetricsCollection.java | 17 ++-- 8 files changed, 162 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/eb40ea57/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java new file mode 100644 index 000..be3b501 --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec.spark; + +import com.google.common.collect.Lists; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistic; +import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; +import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; +import org.apache.hadoop.hive.ql.session.SessionState; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class TestSparkStatistics { + + @Test + public void testSparkStatistics() { +HiveConf conf = new HiveConf(); +conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, +SQLStdHiveAuthorizerFactory.class.getName()); +conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); +conf.setVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "spark"); +conf.set("spark.master", "local-cluster[1,2,1024]"); + +SessionState.start(conf); + +Driver driver = null; + +try { + driver = new Driver(new QueryState.Builder() + .withGenerateNewQueryId(true) + .withHiveConf(conf).build(), + null, null); + + Assert.assertEquals(0, driver.run("create table test (col int)").getResponseCode()); + Assert.assertEquals(0, driver.compile("select * from test order by col")); + + List sparkTasks = Utilities.getSparkTasks(driver.getPlan().getRootTasks()); + Assert.assertEquals(1, sparkTasks.size()); + + SparkTask sparkTask = sparkTasks.get(0); + + DriverContext driverCxt = new DriverContext(driver.getContext()); + driverCxt.prepare(driver.getPlan()); + + sparkTask.initialize(driver.getQueryState(), driver.getPlan(), driverCxt, driver.getContext() + .getOpContext()); + Assert.assertEquals(0, sparkTask.execute(driverCxt)); + +
hive git commit: HIVE-18830: RemoteSparkJobMonitor failures are logged twice (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 95a1538ae -> 6c194473c HIVE-18830: RemoteSparkJobMonitor failures are logged twice (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6c194473 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6c194473 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6c194473 Branch: refs/heads/master Commit: 6c194473c7d7b268781a0f293818736f47ed5756 Parents: 95a1538 Author: Bharathkrishna Guruvayoor MuraliAuthored: Tue Mar 27 09:35:30 2018 -0700 Committer: Sahil Takiar Committed: Tue Mar 27 09:35:30 2018 -0700 -- .../hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/6c194473/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index 3467ae4..a132f74 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -198,7 +198,6 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { // Has to use full name to make sure it does not conflict with // org.apache.commons.lang.StringUtils - LOG.error(msg, e); console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); } rc = 1;
hive git commit: HIVE-18093: Improve logging when HoS application is killed (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 5cb8867bc -> 560bca0df HIVE-18093: Improve logging when HoS application is killed (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/560bca0d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/560bca0d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/560bca0d Branch: refs/heads/master Commit: 560bca0dff92c8e2df9a8d6c31db44404bfa6c1d Parents: 5cb8867 Author: Sahil TakiarAuthored: Thu Mar 22 10:36:39 2018 -0700 Committer: Sahil Takiar Committed: Thu Mar 22 10:36:39 2018 -0700 -- .../apache/hive/spark/client/BaseProtocol.java | 8 ++--- .../apache/hive/spark/client/RemoteDriver.java | 36 +--- 2 files changed, 34 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/560bca0d/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java b/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java index 7ca89ed..4dbc490 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java @@ -49,12 +49,8 @@ abstract class BaseProtocol extends RpcDispatcher { final String cause; -Error(Throwable cause) { - if (cause == null) { -this.cause = ""; - } else { -this.cause = Throwables.getStackTraceAsString(cause); - } +Error(String cause) { + this.cause = cause; } Error() { http://git-wip-us.apache.org/repos/asf/hive/blob/560bca0d/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java index f221d0a..255c305 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java @@ -96,6 +96,8 @@ public class RemoteDriver { this.shutdownLock = new Object(); localTmpDir = Files.createTempDir(); +addShutdownHook(); + SparkConf conf = new SparkConf(); String serverAddress = null; int serverPort = -1; @@ -178,6 +180,17 @@ public class RemoteDriver { } } + private void addShutdownHook() { +Runtime.getRuntime().addShutdownHook(new Thread(() -> { + if (running) { +LOG.info("Received signal SIGTERM, attempting safe shutdown of Remote Spark Context"); +protocol.sendErrorMessage("Remote Spark Context was shutdown because it received a SIGTERM " + +"signal. Most likely due to a kill request via YARN."); +shutdown(null); + } +})); + } + private void run() throws InterruptedException { synchronized (shutdownLock) { while (running) { @@ -249,7 +262,12 @@ public class RemoteDriver { void sendError(Throwable error) { LOG.debug("Send error to Client: {}", Throwables.getStackTraceAsString(error)); - clientRpc.call(new Error(error)); + clientRpc.call(new Error(Throwables.getStackTraceAsString(error))); +} + +void sendErrorMessage(String cause) { + LOG.debug("Send error to Client: {}", cause); + clientRpc.call(new Error(cause)); } void jobFinished(String jobId, T result, @@ -390,7 +408,7 @@ public class RemoteDriver { // Catch throwables in a best-effort to report job status back to the client. It's // re-thrown so that the executor can destroy the affected thread (or the JVM can // die or whatever would happen if the throwable bubbled up). -LOG.info("Failed to run job " + req.id, t); +LOG.error("Failed to run job " + req.id, t); protocol.jobFinished(req.id, null, t, sparkCounters != null ? sparkCounters.snapshot() : null); throw new ExecutionException(t); @@ -515,8 +533,18 @@ public class RemoteDriver { } public static void main(String[] args) throws Exception { -new RemoteDriver(args).run(); +RemoteDriver rd = new RemoteDriver(args); +try { + rd.run(); +} catch (Exception e) { + // If the main thread throws an exception for some reason, propagate the exception to the + // client and initiate a safe shutdown + if (rd.running) { +rd.protocol.sendError(e); +rd.shutdown(null); + } + throw e; +} } - }
hive git commit: HIVE-18344: Remove LinkedList from SharedWorkOptimizer.java (BELUGA BEHR, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master b7c1aa63b -> 79e88695c HIVE-18344: Remove LinkedList from SharedWorkOptimizer.java (BELUGA BEHR, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/79e88695 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/79e88695 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/79e88695 Branch: refs/heads/master Commit: 79e88695cfa47749435e7e793c73697cc3f4a90e Parents: b7c1aa6 Author: BELUGA BEHRAuthored: Mon Mar 19 08:53:45 2018 -0700 Committer: Sahil Takiar Committed: Mon Mar 19 08:53:45 2018 -0700 -- .../apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java| 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/79e88695/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java index bc6d0bb..b60512b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java @@ -27,7 +27,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -595,7 +594,7 @@ public class SharedWorkOptimizer extends Transform { } } List > sortedTables = -new LinkedList<>(tableToTotalSize.entrySet()); +new ArrayList<>(tableToTotalSize.entrySet()); Collections.sort(sortedTables, Collections.reverseOrder( new Comparator >() { @Override @@ -635,7 +634,7 @@ public class SharedWorkOptimizer extends Transform { StatsUtils.safeMult(op.getChildOperators().size(), size)); } List > sortedOps = -new LinkedList<>(opToTotalSize.entrySet()); +new ArrayList<>(opToTotalSize.entrySet()); Collections.sort(sortedOps, Collections.reverseOrder( new Comparator >() { @Override
hive git commit: HIVE-18342: Remove LinkedList from HiveAlterHandler.java (BELUGA BEHR, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master d30d37dbc -> b7c1aa63b HIVE-18342: Remove LinkedList from HiveAlterHandler.java (BELUGA BEHR, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b7c1aa63 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b7c1aa63 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b7c1aa63 Branch: refs/heads/master Commit: b7c1aa63b989f7d9a7fdda20e45ef991d9c5298b Parents: d30d37d Author: BELUGA BEHRAuthored: Mon Mar 19 08:51:53 2018 -0700 Committer: Sahil Takiar Committed: Mon Mar 19 08:51:53 2018 -0700 -- .../org/apache/hadoop/hive/metastore/HiveAlterHandler.java | 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b7c1aa63/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java -- diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java index 0dd3eb1..04828e5 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java +++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java @@ -57,7 +57,6 @@ import java.net.URI; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -272,9 +271,10 @@ public class HiveAlterHandler implements AlterHandler { while (partsToProcess > 0) { int batchEnd = Math.min(batchStart + partitionBatchSize, parts.size()); List partBatch = parts.subList(batchStart, batchEnd); - partsToProcess -= partBatch.size(); - batchStart += partBatch.size(); - List partValues = new LinkedList<>(); + int partBatchSize = partBatch.size(); + partsToProcess -= partBatchSize; + batchStart += partBatchSize; + List
partValues = new ArrayList<>(partBatchSize); for (Partition part : partBatch) { partValues.add(part.getValues()); }
hive git commit: HIVE-16858: Acumulo Utils Improvements (BELUGA BEHR, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 94152c997 -> d30d37dbc HIVE-16858: Acumulo Utils Improvements (BELUGA BEHR, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d30d37db Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d30d37db Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d30d37db Branch: refs/heads/master Commit: d30d37dbccf1082c05370e8960caddc906b56489 Parents: 94152c9 Author: BELUGA BEHRAuthored: Mon Mar 19 08:48:59 2018 -0700 Committer: Sahil Takiar Committed: Mon Mar 19 08:48:59 2018 -0700 -- .../org/apache/hadoop/hive/accumulo/Utils.java | 35 +--- 1 file changed, 15 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d30d37db/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/Utils.java -- diff --git a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/Utils.java b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/Utils.java index 40ec9f2..3a2facf 100644 --- a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/Utils.java +++ b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/Utils.java @@ -40,11 +40,12 @@ import java.util.zip.ZipEntry; import java.util.zip.ZipFile; import java.util.zip.ZipOutputStream; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,9 +72,9 @@ public class Utils { // Add jars containing the specified classes for (Class clazz : classes) { - if (clazz == null) + if (clazz == null) { continue; - + } Path path = findOrCreateJar(clazz, localFs, packagedClasses); if (path == null) { log.warn("Could not find jar for class " + clazz + " in order to ship it to the cluster."); @@ -85,10 +86,9 @@ public class Utils { } jars.add(path.toString()); } -if (jars.isEmpty()) - return; - -conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[jars.size()]))); +if (!jars.isEmpty()) { + conf.set("tmpjars", StringUtils.join(jars, ",")); +} } /** @@ -113,16 +113,16 @@ public class Utils { Map packagedClasses) throws IOException { // attempt to locate an existing jar for the class. String jar = findContainingJar(my_class, packagedClasses); -if (null == jar || jar.isEmpty()) { +if (StringUtils.isEmpty(jar)) { jar = getJar(my_class); updateMap(jar, packagedClasses); } -if (null == jar || jar.isEmpty()) { +if (StringUtils.isEmpty(jar)) { return null; } -log.debug(String.format("For class %s, using jar %s", my_class.getName(), jar)); +log.debug("For class {}, using jar {}", my_class.getName(), jar); return new Path(jar).makeQualified(fs); } @@ -136,7 +136,7 @@ public class Utils { * map[class -> jar] */ private static void updateMap(String jar, Map packagedClasses) throws IOException { -if (null == jar || jar.isEmpty()) { +if (StringUtils.isEmpty(jar)) { return; } ZipFile zip = null; @@ -209,9 +209,9 @@ public class Utils { String hadoopJarFinder = "org.apache.hadoop.util.JarFinder"; Class jarFinder = null; try { - log.debug("Looking for " + hadoopJarFinder + "."); + log.debug("Looking for: {}", hadoopJarFinder); jarFinder = JavaUtils.loadClass(hadoopJarFinder); - log.debug(hadoopJarFinder + " found."); + log.debug("Found: {}", hadoopJarFinder); Method getJar = jarFinder.getMethod("getJar", Class.class); ret = (String) getJar.invoke(null, my_class); } catch (ClassNotFoundException e) { @@ -281,12 +281,7 @@ public class Utils { private static void copyToZipStream(InputStream is, ZipEntry entry, ZipOutputStream zos) throws IOException { zos.putNextEntry(entry); -byte[] arr = new byte[4096]; -int read = is.read(arr); -while (read > -1) { - zos.write(arr, 0, read); - read = is.read(arr); -} +IOUtils.copy(is, zos); is.close(); zos.closeEntry(); } @@ -350,6 +345,6 @@ public class Utils { } } JarOutputStream zos = new JarOutputStream(new FileOutputStream(jarFile)); -jarDir(dir, "", zos); +jarDir(dir, StringUtils.EMPTY, zos); } }
hive git commit: HIVE-18034: Improving logging with HoS executors spend lots of time in GC (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master 9cdc08580 -> 57a1ec211 HIVE-18034: Improving logging with HoS executors spend lots of time in GC (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/57a1ec21 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/57a1ec21 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/57a1ec21 Branch: refs/heads/master Commit: 57a1ec211039d5a5c0eb309adb991283b112520e Parents: 9cdc085 Author: Sahil TakiarAuthored: Wed Mar 14 09:49:02 2018 -0700 Committer: Sahil Takiar Committed: Wed Mar 14 09:49:02 2018 -0700 -- .../hadoop/hive/ql/exec/spark/SparkTask.java| 73 +--- .../spark/Statistic/SparkStatisticGroup.java| 18 +++-- .../exec/spark/Statistic/SparkStatistics.java | 21 -- .../spark/Statistic/SparkStatisticsBuilder.java | 8 +-- .../spark/Statistic/SparkStatisticsNames.java | 43 .../spark/status/impl/JobMetricsListener.java | 24 +++ .../spark/status/impl/LocalSparkJobStatus.java | 19 ++--- .../spark/status/impl/RemoteSparkJobStatus.java | 21 +++--- .../spark/status/impl/SparkMetricsUtils.java| 48 + .../hadoop/hive/ql/history/HiveHistory.java | 1 + .../hive/spark/client/MetricsCollection.java| 4 +- .../apache/hive/spark/client/RemoteDriver.java | 2 +- .../hive/spark/client/metrics/Metrics.java | 14 ++-- .../spark/client/TestMetricsCollection.java | 13 ++-- 14 files changed, 212 insertions(+), 97 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/57a1ec21/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index 76f6ecc..c240884 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -27,9 +27,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import com.google.common.base.Strings; import com.google.common.base.Throwables; import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; +import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -77,7 +79,10 @@ public class SparkTask extends Task { private static final LogHelper console = new LogHelper(LOG); private PerfLogger perfLogger; private static final long serialVersionUID = 1L; + // The id of the actual Spark job private transient int sparkJobID; + // The id of the JobHandle used to track the actual Spark job + private transient String sparkJobHandleId; private transient SparkStatistics sparkStatistics; private transient long submitTime; private transient long startTime; @@ -111,36 +116,60 @@ public class SparkTask extends Task { SparkWork sparkWork = getWork(); sparkWork.setRequiredCounterPrefix(getOperatorCounters()); + // Submit the Spark job perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); submitTime = perfLogger.getStartTime(PerfLogger.SPARK_SUBMIT_JOB); jobRef = sparkSession.submit(driverContext, sparkWork); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); + // If the driver context has been shutdown (due to query cancellation) kill the Spark job if (driverContext.isShutdown()) { LOG.warn("Killing Spark job"); killJob(); throw new HiveException("Operation is cancelled."); } - addToHistory(jobRef); - this.jobID = jobRef.getSparkJobStatus().getAppID(); + // Get the Job Handle id associated with the Spark job + sparkJobHandleId = jobRef.getJobId(); + + // Add Spark job handle id to the Hive History + addToHistory(Keys.SPARK_JOB_HANDLE_ID, jobRef.getJobId()); + + LOG.debug("Starting Spark job with job handle id " + sparkJobHandleId); + + // Get the application id of the Spark app + jobID = jobRef.getSparkJobStatus().getAppID(); + + // Start monitoring the Spark job, returns when the Spark job has completed / failed, or if + // a timeout occurs rc = jobRef.monitorJob(); + + // Get the id the Spark job that was launched, returns -1 if no Spark job was launched + sparkJobID = jobRef.getSparkJobStatus().getJobId(); + + // Add Spark job id to the Hive History + addToHistory(Keys.SPARK_JOB_ID, Integer.toString(sparkJobID)); + +
hive git commit: HIVE-18343: Remove LinkedList from ColumnStatsSemanticAnalyzer.java (BELUGA BEHR, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master db4fe384f -> 9cdc08580 HIVE-18343: Remove LinkedList from ColumnStatsSemanticAnalyzer.java (BELUGA BEHR, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9cdc0858 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9cdc0858 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9cdc0858 Branch: refs/heads/master Commit: 9cdc08580ebecd77f85169b70e1529e3be35db8c Parents: db4fe38 Author: BELUGA BEHRAuthored: Wed Mar 14 09:45:55 2018 -0700 Committer: Sahil Takiar Committed: Wed Mar 14 09:45:55 2018 -0700 -- .../ql/parse/ColumnStatsSemanticAnalyzer.java | 112 +-- 1 file changed, 54 insertions(+), 58 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/9cdc0858/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java index c97e2a9..2780be2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java @@ -20,12 +20,9 @@ package org.apache.hadoop.hive.ql.parse; import java.io.IOException; import java.util.ArrayList; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.common.HiveStatsUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; @@ -45,6 +42,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * ColumnStatsSemanticAnalyzer. * Handles semantic analysis and rewrite for gathering column statistics both at the level of a @@ -54,7 +54,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer { private static final Logger LOG = LoggerFactory .getLogger(ColumnStatsSemanticAnalyzer.class); - static final private LogHelper console = new LogHelper(LOG); + private static final LogHelper CONSOLE = new LogHelper(LOG); private ASTNode originalTree; private ASTNode rewrittenTree; @@ -90,25 +90,25 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer { return rwt; } - private List getColumnName(ASTNode tree) throws SemanticException{ + private List getColumnName(ASTNode tree) throws SemanticException { switch (tree.getChildCount()) { - case 2: - return Utilities.getColumnNamesFromFieldSchema(tbl.getCols()); - case 3: -int numCols = tree.getChild(2).getChildCount(); -List colName = new LinkedList(); -for (int i = 0; i < numCols; i++) { - colName.add(i, new String(getUnescapedName((ASTNode) tree.getChild(2).getChild(i; -} -return colName; - default: -throw new SemanticException("Internal error. Expected number of children of ASTNode to be" -+ " either 2 or 3. Found : " + tree.getChildCount()); +case 2: + return Utilities.getColumnNamesFromFieldSchema(tbl.getCols()); +case 3: + int numCols = tree.getChild(2).getChildCount(); + List colName = new ArrayList(numCols); + for (int i = 0; i < numCols; i++) { +colName.add(getUnescapedName((ASTNode) tree.getChild(2).getChild(i))); + } + return colName; +default: + throw new SemanticException("Internal error. Expected number of children of ASTNode to be" + + " either 2 or 3. Found : " + tree.getChildCount()); } } - private void handlePartialPartitionSpec(Map partSpec, ColumnStatsAutoGatherContext context) throws + private void handlePartialPartitionSpec(Map partSpec, ColumnStatsAutoGatherContext context) throws SemanticException { // If user has fully specified partition, validate that partition exists @@ -133,21 +133,21 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer { // User might have only specified partial list of partition keys, in which case add other partition keys in partSpec List partKeys = Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys()); -for (String partKey : partKeys){ - if(!partSpec.containsKey(partKey)) { - partSpec.put(partKey, null); - } -
hive git commit: HIVE-18901: Lower ResourceDownloader Logging to Debug (Kryvenko Igor, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master ccc82cfb0 -> ed29b4ca9 HIVE-18901: Lower ResourceDownloader Logging to Debug (Kryvenko Igor, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ed29b4ca Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ed29b4ca Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ed29b4ca Branch: refs/heads/master Commit: ed29b4ca91475c24f77a64f2587f4d1972ab9119 Parents: ccc82cf Author: Kryvenko IgorAuthored: Tue Mar 13 09:49:04 2018 -0700 Committer: Sahil Takiar Committed: Tue Mar 13 09:49:04 2018 -0700 -- ql/src/java/org/apache/hadoop/hive/ql/util/ResourceDownloader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ed29b4ca/ql/src/java/org/apache/hadoop/hive/ql/util/ResourceDownloader.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/ResourceDownloader.java b/ql/src/java/org/apache/hadoop/hive/ql/util/ResourceDownloader.java index d74e700..5c5aae5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/util/ResourceDownloader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/util/ResourceDownloader.java @@ -95,7 +95,7 @@ public class ResourceDownloader { private String downloadResource(URI srcUri, String subDir, boolean convertToUnix) throws IOException, URISyntaxException { -LOG.info("converting to local {}", srcUri); +LOG.debug("Converting to local {}", srcUri); File destinationDir = (subDir == null) ? resourceDir : new File(resourceDir, subDir); ensureDirectory(destinationDir); File destinationFile = new File(destinationDir, new Path(srcUri.toString()).getName());
hive git commit: HIVE-18888: Replace synchronizedMap with ConcurrentHashMap (Alexander Kolbasov, reviewed by Peter Vary, Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 12041d39f -> ccc82cfb0 HIVE-1: Replace synchronizedMap with ConcurrentHashMap (Alexander Kolbasov, reviewed by Peter Vary, Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ccc82cfb Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ccc82cfb Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ccc82cfb Branch: refs/heads/master Commit: ccc82cfb0dc1d7c5a0d4a193eb0da434ce8a0cfb Parents: 12041d3 Author: Alexander KolbasovAuthored: Tue Mar 13 09:37:10 2018 -0700 Committer: Sahil Takiar Committed: Tue Mar 13 09:37:10 2018 -0700 -- .../ql/exec/tez/DynamicValueRegistryTez.java | 19 +++ .../org/apache/hadoop/hive/ql/metadata/Hive.java | 14 ++ .../hadoop/hive/metastore/HiveMetaStore.java | 7 +++ 3 files changed, 12 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ccc82cfb/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicValueRegistryTez.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicValueRegistryTez.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicValueRegistryTez.java index 0bed22a..ec1e84b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicValueRegistryTez.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicValueRegistryTez.java @@ -18,35 +18,30 @@ package org.apache.hadoop.hive.ql.exec.tez; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.DynamicValueRegistry; import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; -import org.apache.hadoop.hive.ql.exec.DynamicValueRegistry; -import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.RuntimeValuesInfo; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.DynamicValue.NoDynamicValuesException; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.runtime.api.Input; import org.apache.tez.runtime.api.LogicalInput; import org.apache.tez.runtime.api.ProcessorContext; import org.apache.tez.runtime.library.api.KeyValueReader; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + public class DynamicValueRegistryTez implements DynamicValueRegistry { private static final Logger LOG = LoggerFactory.getLogger(DynamicValueRegistryTez.class); @@ -66,7 +61,7 @@ public class DynamicValueRegistryTez implements DynamicValueRegistry { } } - protected Map values = Collections.synchronizedMap(new HashMap ()); + protected Map values = new ConcurrentHashMap<>(); public DynamicValueRegistryTez() { } http://git-wip-us.apache.org/repos/asf/hive/blob/ccc82cfb/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 6b635fc..5a7e297 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 @@ -37,7 +37,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.LinkedList; @@ -174,13 +173,6 @@ import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Splitter; -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.ThreadFactoryBuilder; - /** * This class has functions that implement meta data/DDL operations using calls * to the metastore. @@ -2064,7 +2056,7 @@ private void
hive git commit: HIVE-16855: org.apache.hadoop.hive.ql.exec.mr.HashTableLoader Improvements (BELUGA BEHR, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 4cb24ce84 -> 0fe38792a HIVE-16855: org.apache.hadoop.hive.ql.exec.mr.HashTableLoader Improvements (BELUGA BEHR, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0fe38792 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0fe38792 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0fe38792 Branch: refs/heads/master Commit: 0fe38792a0852dcfbf58438d3f28257aa332b07f Parents: 4cb24ce Author: BELUGA BEHRAuthored: Mon Mar 12 12:29:00 2018 -0700 Committer: Sahil Takiar Committed: Mon Mar 12 12:29:36 2018 -0700 -- .../hadoop/hive/ql/exec/mr/HashTableLoader.java | 19 +-- 1 file changed, 9 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/0fe38792/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java index 3f3d8f2..3489843 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java @@ -26,6 +26,7 @@ import java.util.List; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.filecache.DistributedCache; import org.apache.hadoop.fs.FileSystem; @@ -53,7 +54,7 @@ import org.apache.hadoop.mapred.JobConf; */ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTableLoader { - private static final Logger LOG = LoggerFactory.getLogger(MapJoinOperator.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(MapJoinOperator.class); private ExecMapperContext context; private Configuration hconf; @@ -76,7 +77,7 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable MapJoinTableContainerSerDe[] mapJoinTableSerdes) throws HiveException { String currentInputPath = context.getCurrentInputPath().toString(); -LOG.info("*** Load from HashTable for input file: " + currentInputPath); +LOG.info("Load from HashTable for input file: {}", currentInputPath); MapredLocalWork localWork = context.getLocalWork(); try { if (localWork.getDirectFetchOp() != null) { @@ -92,9 +93,9 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable continue; } Path path = Utilities.generatePath(baseDir, desc.getDumpFilePrefix(), (byte)pos, fileName); -LOG.info("\tLoad back 1 hashtable file from tmp file uri:" + path); +LOG.info("Load back 1 hashtable file from tmp file uri: {}", path); ObjectInputStream in = new ObjectInputStream(new BufferedInputStream( -new FileInputStream(path.toUri().getPath()), 4096)); +new FileInputStream(path.toUri().getPath(; try{ mapJoinTables[pos] = mapJoinTableSerdes[pos].load(in); } finally { @@ -115,12 +116,10 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable String stageID = localWork.getStageID(); String suffix = Utilities.generateTarFileName(stageID); FileSystem localFs = FileSystem.getLocal(hconf); - for (int j = 0; j < localArchives.length; j++) { -Path archive = localArchives[j]; -if (!archive.getName().endsWith(suffix)) { - continue; + for (Path archive : localArchives) { +if (archive.getName().endsWith(suffix)) { + return archive.makeQualified(localFs); } -return archive.makeQualified(localFs); } } return null; @@ -130,7 +129,7 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable throws Exception { MapredLocalWork localWork = context.getLocalWork(); List directWorks = localWork.getDirectFetchOp().get(joinOp); -if (directWorks == null || directWorks.isEmpty()) { +if (CollectionUtils.isEmpty(directWorks)) { return; } JobConf job = new JobConf(hconf);
hive git commit: HIVE-18917: Add spark.home to hive.conf.restricted.list (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master 41afa4e21 -> dd0bc33d1 HIVE-18917: Add spark.home to hive.conf.restricted.list (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/dd0bc33d Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/dd0bc33d Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/dd0bc33d Branch: refs/heads/master Commit: dd0bc33d125817e20bd4854172b3c8326c834ea0 Parents: 41afa4e Author: Sahil TakiarAuthored: Sat Mar 10 17:55:08 2018 -0800 Committer: Sahil Takiar Committed: Sat Mar 10 17:55:08 2018 -0800 -- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 3 ++- .../src/test/java/org/apache/hive/jdbc/TestRestrictedList.java| 1 + 2 files changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/dd0bc33d/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 04b8c4b..aedd1ec 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3674,7 +3674,8 @@ public class HiveConf extends Configuration { "hive.druid.coordinator.address.default,"+ "hikari.,"+ "hadoop.bin.path,"+ -"yarn.bin.path", +"yarn.bin.path,"+ +"spark.home", "Comma separated list of configuration options which are immutable at runtime"), HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list", METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname http://git-wip-us.apache.org/repos/asf/hive/blob/dd0bc33d/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java index 1ec77b0..79fdb68 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java @@ -106,6 +106,7 @@ public class TestRestrictedList { addToExpectedRestrictedMap("_hive.tmp_table_space"); addToExpectedRestrictedMap("_hive.hdfs.session.path"); addToExpectedRestrictedMap("hive.spark.client.rpc.server.address"); +addToExpectedRestrictedMap("spark.home"); } @AfterClass
hive git commit: HIVE-18436: Upgrade to Spark 2.3.0 (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master b362de387 -> 41afa4e21 HIVE-18436: Upgrade to Spark 2.3.0 (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/41afa4e2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/41afa4e2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/41afa4e2 Branch: refs/heads/master Commit: 41afa4e2185b45b0bb6816baaea330d9ed40d127 Parents: b362de3 Author: Sahil TakiarAuthored: Fri Mar 9 18:54:42 2018 -0800 Committer: Sahil Takiar Committed: Fri Mar 9 18:54:42 2018 -0800 -- itests/pom.xml | 4 ++-- pom.xml| 4 ++-- .../apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java | 2 +- .../org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java | 3 ++- .../test/java/org/apache/hive/spark/client/rpc/TestRpc.java| 6 -- 5 files changed, 11 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/41afa4e2/itests/pom.xml -- diff --git a/itests/pom.xml b/itests/pom.xml index a782cd2..40bb054 100644 --- a/itests/pom.xml +++ b/itests/pom.xml @@ -99,10 +99,10 @@ cd - fi tar -zxf $DOWNLOAD_DIR/$tarName -C $BASE_DIR -mv $BASE_DIR/spark-${spark.version}-bin-hadoop2-without-hive $BASE_DIR/$finalName +mv $BASE_DIR/spark-${spark.version}-bin-hadoop3-beta1-without-hive $BASE_DIR/$finalName } mkdir -p $DOWNLOAD_DIR - download "http://d3jw87u4immizc.cloudfront.net/spark-tarball/spark-${spark.version}-bin-hadoop2-without-hive.tgz; "spark" + download "http://d3jw87u4immizc.cloudfront.net/spark-tarball/spark-${spark.version}-bin-hadoop3-beta1-without-hive.tgz; "spark" cp -f $HIVE_ROOT/data/conf/spark/log4j2.properties $BASE_DIR/spark/conf/ http://git-wip-us.apache.org/repos/asf/hive/blob/41afa4e2/pom.xml -- diff --git a/pom.xml b/pom.xml index a242fbf..5be30f6 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ 1.4.3 1.10.19 2.0.0-M5 -4.0.52.Final +4.1.17.Final 3.10.5.Final 1.9.0 0.16.0 @@ -198,7 +198,7 @@ 0.9.1 0.92.0-incubating 2.2.0 -2.2.0 +2.3.0 2.11 2.11.8 1.1 http://git-wip-us.apache.org/repos/asf/hive/blob/41afa4e2/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index f43b449..dff3b0f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -104,7 +104,7 @@ public class LocalHiveSparkClient implements HiveSparkClient { sc.addJar(regJar); } jobMetricsListener = new JobMetricsListener(); -sc.sc().listenerBus().addListener(jobMetricsListener); +sc.sc().addSparkListener(jobMetricsListener); } @Override http://git-wip-us.apache.org/repos/asf/hive/blob/41afa4e2/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java -- diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java index 24858d7..fb73647 100644 --- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java +++ b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java @@ -72,7 +72,8 @@ public class TestKryoMessageCodec { c.writeAndFlush(MESSAGE); assertEquals(1, c.outboundMessages().size()); assertFalse(MESSAGE.getClass().equals(c.outboundMessages().peek().getClass())); -c.writeInbound(c.readOutbound()); +Object readOutboundResult = c.readOutbound(); +c.writeInbound(readOutboundResult); assertEquals(1, c.inboundMessages().size()); assertEquals(MESSAGE, c.readInbound()); c.close(); http://git-wip-us.apache.org/repos/asf/hive/blob/41afa4e2/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
hive git commit: HIVE-18663: Logged Spark Job Id contains a UUID instead of the actual id (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master e3c4d51a1 -> 571ef51ac HIVE-18663: Logged Spark Job Id contains a UUID instead of the actual id (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/571ef51a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/571ef51a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/571ef51a Branch: refs/heads/master Commit: 571ef51ac22b9828f39acce1a61298ce9d76e966 Parents: e3c4d51 Author: Sahil TakiarAuthored: Fri Feb 23 10:25:11 2018 -0800 Committer: Sahil Takiar Committed: Fri Feb 23 10:25:11 2018 -0800 -- .../apache/hadoop/hive/ql/exec/spark/SparkTask.java | 14 -- .../exec/spark/status/impl/RemoteSparkJobStatus.java | 2 +- 2 files changed, 9 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/571ef51a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index 62d..76f6ecc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -77,7 +77,7 @@ public class SparkTask extends Task { private static final LogHelper console = new LogHelper(LOG); private PerfLogger perfLogger; private static final long serialVersionUID = 1L; - private transient String sparkJobID; + private transient int sparkJobID; private transient SparkStatistics sparkStatistics; private transient long submitTime; private transient long startTime; @@ -123,18 +123,19 @@ public class SparkTask extends Task { } addToHistory(jobRef); - sparkJobID = jobRef.getJobId(); this.jobID = jobRef.getSparkJobStatus().getAppID(); rc = jobRef.monitorJob(); SparkJobStatus sparkJobStatus = jobRef.getSparkJobStatus(); + sparkJobID = sparkJobStatus.getJobId(); getSparkJobInfo(sparkJobStatus, rc); if (rc == 0) { sparkStatistics = sparkJobStatus.getSparkStatistics(); if (LOG.isInfoEnabled() && sparkStatistics != null) { - LOG.info(String.format("=Spark Job[%s] statistics=", jobRef.getJobId())); + LOG.info(String.format("=Spark Job[%s] statistics=", sparkJobID)); logSparkStatistic(sparkStatistics); } -LOG.info("Execution completed successfully"); +LOG.info("Successfully completed Spark Job " + sparkJobID + " with application ID " + +jobID + " and task ID " + getId()); } else if (rc == 2) { // Cancel job if the monitor found job submission timeout. // TODO: If the timeout is because of lack of resources in the cluster, we should // ideally also cancel the app request here. But w/o facilities from Spark or YARN, @@ -192,7 +193,8 @@ public class SparkTask extends Task { console.printInfo("Starting Spark Job = " + jobRef.getJobId()); if (SessionState.get() != null) { SessionState.get().getHiveHistory() - .setQueryProperty(queryState.getQueryId(), Keys.SPARK_JOB_ID, jobRef.getJobId()); + .setQueryProperty(queryState.getQueryId(), Keys.SPARK_JOB_ID, + Integer.toString(jobRef.getSparkJobStatus().getJobId())); } } @@ -277,7 +279,7 @@ public class SparkTask extends Task { return ((ReduceWork) children.get(0)).getReducer(); } - public String getSparkJobID() { + public int getSparkJobID() { return sparkJobID; } http://git-wip-us.apache.org/repos/asf/hive/blob/571ef51a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java index e950452..ec7ca40 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java @@ -129,7 +129,7 @@ public class RemoteSparkJobStatus implements SparkJobStatus { // add Hive operator level statistics. sparkStatisticsBuilder.add(getCounter()); // add spark job metrics. -String jobIdentifier = "Spark Job[" + jobHandle.getClientJobId() + "] Metrics"; +String jobIdentifier = "Spark Job[" + getJobId() + "] Metrics"; Map flatJobMetric = SparkMetricsUtils.collectMetrics(
hive git commit: HIVE-18765: SparkClientImpl swallows exception messages from the RemoteDriver (Sahil Takiar, reviewed by Xuefu Zhang)
Repository: hive Updated Branches: refs/heads/master cbb9233a3 -> e3c4d51a1 HIVE-18765: SparkClientImpl swallows exception messages from the RemoteDriver (Sahil Takiar, reviewed by Xuefu Zhang) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e3c4d51a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e3c4d51a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e3c4d51a Branch: refs/heads/master Commit: e3c4d51a1d49b3398f8c82237c93f8107d71ad2e Parents: cbb9233 Author: Sahil TakiarAuthored: Fri Feb 23 09:59:31 2018 -0800 Committer: Sahil Takiar Committed: Fri Feb 23 09:59:31 2018 -0800 -- .../main/java/org/apache/hive/spark/client/SparkClientImpl.java| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e3c4d51a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java index 214f74b..665ed92 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java @@ -575,7 +575,7 @@ class SparkClientImpl implements SparkClient { } private void handle(ChannelHandlerContext ctx, Error msg) { - LOG.warn("Error reported from remote driver.", msg.cause); + LOG.warn("Error reported from remote driver: {}", msg.cause); } private void handle(ChannelHandlerContext ctx, JobMetrics msg) {
hive git commit: HIVE-18625: SessionState Not Checking For Directory Creation Result (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 03a1e6247 -> 111ed0964 HIVE-18625: SessionState Not Checking For Directory Creation Result (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/111ed096 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/111ed096 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/111ed096 Branch: refs/heads/master Commit: 111ed096496c37674601173cfeaa40cbf948f878 Parents: 03a1e62 Author: Andrew ShermanAuthored: Tue Feb 20 11:28:20 2018 -0800 Committer: Sahil Takiar Committed: Tue Feb 20 11:28:34 2018 -0800 -- .../hadoop/hive/ql/session/SessionState.java| 7 ++- .../hive/ql/session/TestSessionState.java | 53 2 files changed, 58 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/111ed096/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 dfc2dfa..0071a9a 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 @@ -740,7 +740,8 @@ public class SessionState { * @return * @throws IOException */ - private static void createPath(HiveConf conf, Path path, String permission, boolean isLocal, + @VisibleForTesting + static void createPath(HiveConf conf, Path path, String permission, boolean isLocal, boolean isCleanUp) throws IOException { FsPermission fsPermission = new FsPermission(permission); FileSystem fs; @@ -750,7 +751,9 @@ public class SessionState { fs = path.getFileSystem(conf); } if (!fs.exists(path)) { - fs.mkdirs(path, fsPermission); + if (!fs.mkdirs(path, fsPermission)) { +throw new IOException("Failed to create directory " + path + " on fs " + fs.getUri()); + } String dirType = isLocal ? "local" : "HDFS"; LOG.info("Created " + dirType + " directory: " + path.toString()); } http://git-wip-us.apache.org/repos/asf/hive/blob/111ed096/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java -- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java b/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java index 8750196..0fa1c81 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql.session; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; @@ -27,6 +29,10 @@ import java.util.Arrays; import java.util.Collection; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.Warehouse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -257,4 +263,51 @@ public class TestSessionState { } } } + + /** + * Unit test for SessionState.createPath(). + */ + @Test + public void testCreatePath() throws Exception { +HiveConf conf = new HiveConf(); +LocalFileSystem localFileSystem = FileSystem.getLocal(conf); + +Path repeatedCreate = new Path("repeatedCreate"); +SessionState.createPath(conf, repeatedCreate, "700", true, true); +assertTrue(localFileSystem.exists(repeatedCreate)); +// second time will complete silently +SessionState.createPath(conf, repeatedCreate, "700", true, true); + +Path fileNotDirectory = new Path("fileNotDirectory"); +localFileSystem.create(fileNotDirectory); +localFileSystem.deleteOnExit(fileNotDirectory); + +// Show we cannot create a child of a file +try { + SessionState.createPath(conf, new Path(fileNotDirectory, "child"), "700", true, true); + fail("did not get expected exception creating a child of a file"); +} catch (ParentNotDirectoryException e) { + assertTrue(e.getMessage().contains("Parent path is not a directory")); +} + +// Show we cannot create a child of a null directory +try { + //noinspection ConstantConditions + SessionState.createPath(conf, new Path((String) null, "child"), "700", true, true); + fail("did
hive git commit: HIVE-18672: Printed state in RemoteSparkJobMonitor is ambiguous (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 217811254 -> fd561935d HIVE-18672: Printed state in RemoteSparkJobMonitor is ambiguous (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fd561935 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fd561935 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fd561935 Branch: refs/heads/master Commit: fd561935d4d1e34f25bc4c851a8acf37ec7879a8 Parents: 2178112 Author: Sahil TakiarAuthored: Thu Feb 15 15:01:36 2018 -0800 Committer: Sahil Takiar Committed: Thu Feb 15 15:01:36 2018 -0800 -- .../exec/spark/status/RemoteSparkJobMonitor.java | 18 +- .../ql/exec/spark/status/SparkJobMonitor.java | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fd561935/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index adb65a5..3467ae4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -62,10 +62,11 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); startTime = System.currentTimeMillis(); +JobHandle.State state = null; while (true) { try { -JobHandle.State state = sparkJobStatus.getRemoteJobState(); +state = sparkJobStatus.getRemoteJobState(); Preconditions.checkState(sparkJobStatus.isRemoteActive(), "Connection to remote Spark driver was lost"); switch (state) { @@ -76,14 +77,13 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { HiveException he = new HiveException(ErrorMsg.SPARK_JOB_MONITOR_TIMEOUT, Long.toString(timeCount)); console.printError(he.getMessage()); -console.printError("Status: " + state); sparkJobStatus.setError(he); running = false; done = true; rc = 2; } if (LOG.isDebugEnabled()) { -console.printInfo("state = " + state); +console.printInfo("Spark job state = " + state ); } break; case STARTED: @@ -98,8 +98,7 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { console.printInfo("\nQuery Hive on Spark job[" + sparkJobStatus.getJobId() + "] stages: " + Arrays.toString(sparkJobStatus.getStageIds())); - console.printInfo("\nStatus: Running (Hive on Spark job[" -+ sparkJobStatus.getJobId() + "])"); + console.printInfo("Spark job[" + sparkJobStatus.getJobId() + "] status = RUNNING"); running = true; String format = "Job Progress Format\nCurrentTime StageId_StageAttemptId: " @@ -142,8 +141,8 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { printStatus(progressMap, lastProgressMap); lastProgressMap = progressMap; double duration = (System.currentTimeMillis() - startTime) / 1000.0; - console.printInfo("Status: Finished successfully in " -+ String.format("%.2f seconds", duration)); + console.printInfo("Spark job[" + sparkJobStatus.getJobId() + "] finished successfully in " ++ String.format("%.2f second(s)", duration)); running = false; done = true; break; @@ -176,7 +175,7 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { rc = 3; break; case CANCELLED: - console.printInfo("Status: Cancelled"); + console.printInfo("Spark job[" + sparkJobStatus.getJobId() + " was cancelled"); running = false; done = true; rc = 3; @@ -193,7 +192,8 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { finalException = new HiveException(e, ErrorMsg.SPARK_JOB_INTERRUPTED); LOG.warn("Interrupted while monitoring the Hive on Spark application, exiting"); } else { - String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; + String msg = " with exception '" + Utilities.getNameMessage(e) + "' Last known state = " + + (state != null ? state.name() : "UNKNOWN"); msg = "Failed to monitor Job[" +
hive git commit: HIVE-18677: SparkClientImpl usage of SessionState.LogHelper doesn't respect isSilent value (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master a2d22b44d -> 71d6e1652 HIVE-18677: SparkClientImpl usage of SessionState.LogHelper doesn't respect isSilent value (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/71d6e165 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/71d6e165 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/71d6e165 Branch: refs/heads/master Commit: 71d6e16522e1965a2c28d451b0811ce123022d9d Parents: a2d22b4 Author: Sahil TakiarAuthored: Wed Feb 14 16:01:18 2018 -0800 Committer: Sahil Takiar Committed: Wed Feb 14 16:01:18 2018 -0800 -- .../ql/exec/spark/RemoteHiveSparkClient.java| 3 +- .../hive/ql/exec/spark/SparkUtilities.java | 9 ++ .../spark/status/RemoteSparkJobMonitor.java | 1 + .../ql/exec/spark/status/SparkJobStatus.java| 2 ++ .../spark/status/impl/LocalSparkJobStatus.java | 16 ++ .../spark/status/impl/RemoteSparkJobStatus.java | 31 .../apache/hive/spark/client/BaseProtocol.java | 13 .../apache/hive/spark/client/RemoteDriver.java | 13 .../hive/spark/client/SparkClientFactory.java | 4 +-- .../hive/spark/client/SparkClientImpl.java | 10 +-- .../hive/spark/client/TestSparkClient.java | 3 +- 11 files changed, 64 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/71d6e165/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 93d44dc..c571d1a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -101,8 +101,7 @@ public class RemoteHiveSparkClient implements HiveSparkClient { } private void createRemoteClient() throws Exception { -remoteClient = SparkClientFactory.createClient(conf, hiveConf, sessionId, -SessionState.LogHelper.getInfoStream()); +remoteClient = SparkClientFactory.createClient(conf, hiveConf, sessionId); if (HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_PREWARM_ENABLED) && (SparkClientUtilities.isYarnMaster(hiveConf.get("spark.master")) || http://git-wip-us.apache.org/repos/asf/hive/blob/71d6e165/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java index 943a4ee..913c9d7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java @@ -18,7 +18,9 @@ package org.apache.hadoop.hive.ql.exec.spark; import java.io.IOException; +import java.net.InetAddress; import java.net.URI; +import java.net.UnknownHostException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; @@ -304,4 +306,11 @@ public class SparkUtilities { } return false; } + + public static String reverseDNSLookupURL(String url) throws UnknownHostException { +// Run a reverse DNS lookup on the URL +URI uri = URI.create(url); +InetAddress address = InetAddress.getByName(uri.getHost()); +return uri.getScheme() + "://" + address.getCanonicalHostName() + ":" + uri.getPort(); + } } http://git-wip-us.apache.org/repos/asf/hive/blob/71d6e165/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index fc4e4de..adb65a5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -93,6 +93,7 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { if (!running) { perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); printAppInfo(); + console.printInfo("Hive on Spark Session Web UI URL: " + sparkJobStatus.getWebUIURL()); // print job stages. console.printInfo("\nQuery Hive on Spark job[" + sparkJobStatus.getJobId() +
hive git commit: HIVE-18673: ErrorMsg.SPARK_JOB_MONITOR_TIMEOUT isn't formatted correctly (Sahil Takiar, reviewed by Chao Sun)
Repository: hive Updated Branches: refs/heads/master cf4114e1b -> fedefeba6 HIVE-18673: ErrorMsg.SPARK_JOB_MONITOR_TIMEOUT isn't formatted correctly (Sahil Takiar, reviewed by Chao Sun) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/fedefeba Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/fedefeba Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/fedefeba Branch: refs/heads/master Commit: fedefeba65e0ce98fe3e5db5c802982ac70acf53 Parents: cf4114e Author: Sahil TakiarAuthored: Tue Feb 13 16:07:44 2018 -0800 Committer: Sahil Takiar Committed: Tue Feb 13 16:07:44 2018 -0800 -- ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/fedefeba/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 46d876d..39a613c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -557,10 +557,10 @@ public enum ErrorMsg { SPARK_GET_JOB_INFO_TIMEOUT(30036, "Spark job timed out after {0} seconds while getting job info", true), - SPARK_JOB_MONITOR_TIMEOUT(30037, "Job hasn't been submitted after {0}s." + + SPARK_JOB_MONITOR_TIMEOUT(30037, "Job hasn''t been submitted after {0}s." + " Aborting it.\nPossible reasons include network issues, " + "errors in remote driver or the cluster has no available resources, etc.\n" + - "Please check YARN or Spark driver's logs for further information.\n" + + "Please check YARN or Spark driver''s logs for further information.\n" + "The timeout is controlled by " + HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT + ".", true), // Various errors when creating Spark client
hive git commit: HIVE-17837: Explicitly check if the HoS Remote Driver has been lost in the RemoteSparkJobMonitor (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master e33edd964 -> 89e75c785 HIVE-17837: Explicitly check if the HoS Remote Driver has been lost in the RemoteSparkJobMonitor (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/89e75c78 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/89e75c78 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/89e75c78 Branch: refs/heads/master Commit: 89e75c78524327ef0c6111b4d90504f3bda781d4 Parents: e33edd9 Author: Sahil TakiarAuthored: Fri Feb 9 15:03:15 2018 -0800 Committer: Sahil Takiar Committed: Fri Feb 9 15:03:15 2018 -0800 -- .../hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java | 5 + 1 file changed, 1 insertion(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/89e75c78/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index 4c4ce55..22f7024 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -66,6 +66,7 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { while (true) { try { JobHandle.State state = sparkJobStatus.getRemoteJobState(); +Preconditions.checkState(sparkJobStatus.isRemoteActive(), "Connection to remote Spark driver was lost"); switch (state) { case SENT: @@ -133,10 +134,6 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { printStatus(progressMap, lastProgressMap); lastProgressMap = progressMap; - } else if (sparkJobState == null) { -// in case the remote context crashes between JobStarted and JobSubmitted -Preconditions.checkState(sparkJobStatus.isRemoteActive(), -"Remote context becomes inactive."); } break; case SUCCEEDED:
hive git commit: HIVE-17835: HS2 Logs print unnecessary stack trace when HoS query is cancelled (Sahil Takiar, reviewed by Chao Sun)
Repository: hive Updated Branches: refs/heads/master 717ef18d9 -> e33edd964 HIVE-17835: HS2 Logs print unnecessary stack trace when HoS query is cancelled (Sahil Takiar, reviewed by Chao Sun) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e33edd96 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e33edd96 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e33edd96 Branch: refs/heads/master Commit: e33edd9649ce05495396a2183b1be3d1a79fd0d3 Parents: 717ef18 Author: Sahil TakiarAuthored: Fri Feb 9 14:49:38 2018 -0800 Committer: Sahil Takiar Committed: Fri Feb 9 14:49:38 2018 -0800 -- .../hadoop/hive/ql/exec/spark/SparkTask.java | 4 +++- .../exec/spark/status/RemoteSparkJobMonitor.java | 19 +++ 2 files changed, 14 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e33edd96/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index c6e17b5..62d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -117,6 +117,7 @@ public class SparkTask extends Task { perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); if (driverContext.isShutdown()) { +LOG.warn("Killing Spark job"); killJob(); throw new HiveException("Operation is cancelled."); } @@ -337,7 +338,7 @@ public class SparkTask extends Task { try { jobRef.cancelJob(); } catch (Exception e) { -LOG.warn("failed to kill job", e); +LOG.warn("Failed to kill Spark job", e); } } } @@ -424,6 +425,7 @@ public class SparkTask extends Task { if ((error instanceof InterruptedException) || (error instanceof HiveException && error.getCause() instanceof InterruptedException)) { +LOG.info("Killing Spark job since query was interrupted"); killJob(); } HiveException he; http://git-wip-us.apache.org/repos/asf/hive/blob/e33edd96/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java index 6c7aca7..4c4ce55 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java @@ -184,16 +184,19 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor { } } catch (Exception e) { Exception finalException = e; -if (e instanceof InterruptedException) { +if (e instanceof InterruptedException || +(e instanceof HiveException && e.getCause() instanceof InterruptedException)) { finalException = new HiveException(e, ErrorMsg.SPARK_JOB_INTERRUPTED); + LOG.warn("Interrupted while monitoring the Hive on Spark application, exiting"); +} else { + String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; + msg = "Failed to monitor Job[" + sparkJobStatus.getJobId() + "]" + msg; + + // Has to use full name to make sure it does not conflict with + // org.apache.commons.lang.StringUtils + LOG.error(msg, e); + console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); } -String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; -msg = "Failed to monitor Job[" + sparkJobStatus.getJobId() + "]" + msg; - -// Has to use full name to make sure it does not conflict with -// org.apache.commons.lang.StringUtils -LOG.error(msg, e); -console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); rc = 1; done = true; sparkJobStatus.setError(finalException);
hive git commit: HIVE-18368: Improve Spark Debug RDD Graph (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master 43e713746 -> 1e74aca8d HIVE-18368: Improve Spark Debug RDD Graph (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1e74aca8 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1e74aca8 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1e74aca8 Branch: refs/heads/master Commit: 1e74aca8d09ea2ef636311d2168b4d34198f7194 Parents: 43e7137 Author: Sahil TakiarAuthored: Thu Feb 8 12:45:58 2018 -0800 Committer: Sahil Takiar Committed: Thu Feb 8 12:45:58 2018 -0800 -- .../hadoop/hive/ql/exec/spark/CacheTran.java| 14 +- .../ql/exec/spark/LocalHiveSparkClient.java | 6 + .../hadoop/hive/ql/exec/spark/MapInput.java | 13 +- .../hadoop/hive/ql/exec/spark/MapTran.java | 17 +- .../hadoop/hive/ql/exec/spark/ReduceTran.java | 17 +- .../hadoop/hive/ql/exec/spark/ShuffleTran.java | 19 ++- .../hadoop/hive/ql/exec/spark/SparkPlan.java| 164 ++- .../hive/ql/exec/spark/SparkPlanGenerator.java | 35 +++- .../hadoop/hive/ql/exec/spark/SparkTran.java| 2 - .../hive/ql/exec/spark/SparkUtilities.java | 36 +--- .../hive/ql/io/CombineHiveInputFormat.java | 2 +- 11 files changed, 85 insertions(+), 240 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/1e74aca8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java index c5fec7d..4b77ac9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java @@ -27,9 +27,11 @@ public abstract class CacheTran cachedRDD; + protected final String name; - protected CacheTran(boolean cache) { + protected CacheTran(boolean cache, String name) { this.caching = cache; +this.name = name; } @Override @@ -40,9 +42,10 @@ public abstract class CacheTran rdd = doTransform(input); + return rdd.setName(this.name + " (" + rdd.getNumPartitions() + ")"); } } @@ -51,4 +54,9 @@ public abstract class CacheTran doTransform(JavaPairRDD input); + + @Override + public String getName() { +return name; + } } http://git-wip-us.apache.org/repos/asf/hive/blob/1e74aca8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index cab97a0..f43b449 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -25,7 +25,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import org.apache.hadoop.hive.ql.exec.DagUtils; import org.apache.hive.spark.client.SparkClientUtilities; +import org.apache.spark.util.CallSite; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileSystem; @@ -160,8 +162,12 @@ public class LocalHiveSparkClient implements HiveSparkClient { // Execute generated plan. JavaPairRDD finalRDD = plan.generateGraph(); + +sc.setJobGroup("queryId = " + sparkWork.getQueryId(), DagUtils.getQueryName(jobConf)); + // We use Spark RDD async action to submit job as it's the only way to get jobId now. JavaFutureAction future = finalRDD.foreachAsync(HiveVoidFunction.getInstance()); + // As we always use foreach action to submit RDD graph, it would only trigger one job. int jobId = future.jobIds().get(0); LocalSparkJobStatus sparkJobStatus = new LocalSparkJobStatus( http://git-wip-us.apache.org/repos/asf/hive/blob/1e74aca8/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java index d240d18..b1a0d55 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java @@ -36,17 +36,18 @@ public class MapInput implements SparkTran hadoopRDD; private boolean toCache; private final SparkPlan sparkPlan; - private String name = "MapInput"; + private final String
hive git commit: HIVE-18389: Print out Spark Web UI URL to the console log (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master b8fdd1392 -> 43e713746 HIVE-18389: Print out Spark Web UI URL to the console log (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/43e71374 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/43e71374 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/43e71374 Branch: refs/heads/master Commit: 43e713746f038fb55e4a63dba59945bbfe448471 Parents: b8fdd13 Author: Sahil TakiarAuthored: Thu Feb 8 11:32:26 2018 -0800 Committer: Sahil Takiar Committed: Thu Feb 8 11:32:26 2018 -0800 -- .../hive/ql/exec/spark/RemoteHiveSparkClient.java| 3 ++- .../org/apache/hive/spark/client/BaseProtocol.java | 12 .../org/apache/hive/spark/client/RemoteDriver.java | 15 +++ .../apache/hive/spark/client/SparkClientFactory.java | 8 +--- .../apache/hive/spark/client/SparkClientImpl.java| 10 +- .../apache/hive/spark/client/TestSparkClient.java| 4 +++- 6 files changed, 46 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/43e71374/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index c571d1a..93d44dc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -101,7 +101,8 @@ public class RemoteHiveSparkClient implements HiveSparkClient { } private void createRemoteClient() throws Exception { -remoteClient = SparkClientFactory.createClient(conf, hiveConf, sessionId); +remoteClient = SparkClientFactory.createClient(conf, hiveConf, sessionId, +SessionState.LogHelper.getInfoStream()); if (HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_PREWARM_ENABLED) && (SparkClientUtilities.isYarnMaster(hiveConf.get("spark.master")) || http://git-wip-us.apache.org/repos/asf/hive/blob/43e71374/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java b/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java index 0930325..7290809 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java @@ -166,4 +166,16 @@ abstract class BaseProtocol extends RpcDispatcher { } + protected static class SparkUIWebURL implements Serializable { + +final String UIWebURL; + +SparkUIWebURL(String UIWebURL) { + this.UIWebURL = UIWebURL; +} + +SparkUIWebURL() { + this(null); +} + } } http://git-wip-us.apache.org/repos/asf/hive/blob/43e71374/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java index ede8ce9..66cf339 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java @@ -25,6 +25,8 @@ import io.netty.channel.nio.NioEventLoopGroup; import java.io.File; import java.io.IOException; import java.io.Serializable; +import java.net.InetAddress; +import java.net.URI; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -169,6 +171,14 @@ public class RemoteDriver { throw e; } +if (jc.sc().sc().uiWebUrl().isDefined()) { + // Run a reverse DNS lookup on the URL + URI uri = URI.create(jc.sc().sc().uiWebUrl().get()); + InetAddress address = InetAddress.getByName(uri.getHost()); + this.protocol.sendUIWebURL(uri.getScheme() + "://" + address.getCanonicalHostName() + ":" + + uri.getPort()); +} + synchronized (jcLock) { for (Iterator it = jobQueue.iterator(); it.hasNext();) { it.next().submit(); @@ -270,6 +280,11 @@ public class RemoteDriver { clientRpc.call(new JobMetrics(jobId, sparkJobId, stageId, taskId, metrics)); } +void sendUIWebURL(String UIWebURL) { + LOG.debug("Send UIWebURL({}) to Client.", UIWebURL); + clientRpc.call(new SparkUIWebURL(UIWebURL)); +} + private void
hive git commit: HIVE-18639: HiveMetaStoreFsImpl#deleteDir should check return value of FileUtils#moveToTrash (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master cf3394de0 -> 075077d3c HIVE-18639: HiveMetaStoreFsImpl#deleteDir should check return value of FileUtils#moveToTrash (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/075077d3 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/075077d3 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/075077d3 Branch: refs/heads/master Commit: 075077d3c50ca24826aa76b06c041b949bfbc002 Parents: cf3394d Author: Sahil TakiarAuthored: Wed Feb 7 12:53:30 2018 -0800 Committer: Sahil Takiar Committed: Wed Feb 7 12:53:44 2018 -0800 -- .../org/apache/hadoop/hive/metastore/HiveMetaStoreFsImpl.java| 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/075077d3/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreFsImpl.java -- diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreFsImpl.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreFsImpl.java index 8322cb8..03de6c0 100644 --- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreFsImpl.java +++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreFsImpl.java @@ -38,7 +38,9 @@ public class HiveMetaStoreFsImpl implements MetaStoreFS { public boolean deleteDir(FileSystem fs, Path f, boolean recursive, boolean ifPurge, Configuration conf) throws MetaException { try { - FileUtils.moveToTrash(fs, f, conf, ifPurge); + if (FileUtils.moveToTrash(fs, f, conf, ifPurge)) { +return true; + } if (fs.exists(f)) { throw new MetaException("Unable to delete directory: " + f); }
hive git commit: HIVE-18596: Synchronize value of hive.spark.client.connect.timeout across unit tests (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 3a677303f -> e0eee6bef HIVE-18596: Synchronize value of hive.spark.client.connect.timeout across unit tests (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e0eee6be Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e0eee6be Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e0eee6be Branch: refs/heads/master Commit: e0eee6bef3fa5816b8ca60b7e737a12a99dcecfa Parents: 3a67730 Author: Sahil TakiarAuthored: Mon Feb 5 11:43:26 2018 -0600 Committer: Sahil Takiar Committed: Mon Feb 5 11:44:03 2018 -0600 -- data/conf/spark/standalone/hive-site.xml| 5 + .../org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java | 1 + .../hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java| 1 + .../test/java/org/apache/hive/spark/client/TestSparkClient.java | 4 4 files changed, 11 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/e0eee6be/data/conf/spark/standalone/hive-site.xml -- diff --git a/data/conf/spark/standalone/hive-site.xml b/data/conf/spark/standalone/hive-site.xml index 1e5bd65..7095979 100644 --- a/data/conf/spark/standalone/hive-site.xml +++ b/data/conf/spark/standalone/hive-site.xml @@ -255,6 +255,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.spark.client.connect.timeout + 3ms + + hive.stats.column.autogather http://git-wip-us.apache.org/repos/asf/hive/blob/e0eee6be/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java index 0fee3c6..2a4da20 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java @@ -70,6 +70,7 @@ public class TestJdbcWithLocalClusterSpark { conf.set("hive.execution.engine", "spark"); conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.set("spark.master", "local-cluster[2,2,1024]"); +conf.set("hive.spark.client.connect.timeout", "3ms"); // FIXME: Hadoop3 made the incompatible change for dfs.client.datanode-restart.timeout // while spark2 is still using Hadoop2. // Spark requires Hive to support Hadoop3 first then Spark can start http://git-wip-us.apache.org/repos/asf/hive/blob/e0eee6be/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java index 93fe776..9f72e51 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java @@ -79,6 +79,7 @@ public class TestMultiSessionsHS2WithLocalClusterSpark { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.set("spark.master", "local-cluster[2,2,1024]"); conf.set("spark.deploy.defaultCores", "2"); +conf.set("hive.spark.client.connect.timeout", "3ms"); // FIXME: Hadoop3 made the incompatible change for dfs.client.datanode-restart.timeout // while spark2 is still using Hadoop2. // Spark requires Hive to support Hadoop3 first then Spark can start http://git-wip-us.apache.org/repos/asf/hive/blob/e0eee6be/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java -- diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java index 23df792..579ca07 100644 --- a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java +++ b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java @@ -68,6 +68,10 @@ public class TestSparkClient { private static final long TIMEOUT = 20; private static final HiveConf HIVECONF = new HiveConf(); + static { +HIVECONF.set("hive.spark.client.connect.timeout", "3ms"); + } + private Map
hive git commit: HIVE-18552: Split hive.strict.checks.large.query into two configs (Sahil Takiar, reviewed by Vihang Karajgaonkar)
Repository: hive Updated Branches: refs/heads/master 47f45705f -> 4a33ec8fc HIVE-18552: Split hive.strict.checks.large.query into two configs (Sahil Takiar, reviewed by Vihang Karajgaonkar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4a33ec8f Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4a33ec8f Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4a33ec8f Branch: refs/heads/master Commit: 4a33ec8fcae5f7d18105ef62e33150db6e853af5 Parents: 47f4570 Author: Sahil TakiarAuthored: Fri Feb 2 20:34:32 2018 -0600 Committer: Sahil Takiar Committed: Fri Feb 2 20:34:32 2018 -0600 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 21 +++- .../clientnegative/alter_view_failure6_2.q | 2 +- .../queries/clientnegative/input_part0_neg_2.q | 2 +- .../queries/clientnegative/strict_orderby_2.q | 8 .../queries/clientnegative/strict_pruning_2.q | 4 +--- .../clientnegative/alter_view_failure6.q.out| 4 ++-- .../clientnegative/alter_view_failure6_2.q.out | 4 ++-- .../clientnegative/compare_double_bigint.q.out | 2 +- .../compare_double_bigint_2.q.out | 2 +- .../clientnegative/compare_string_bigint.q.out | 2 +- .../compare_string_bigint_2.q.out | 2 +- ql/src/test/results/clientnegative/input4.q.out | 2 +- .../test/results/clientnegative/input4_2.q.out | 2 +- .../clientnegative/input_part0_neg.q.out| 2 +- .../clientnegative/input_part0_neg_2.q.out | 2 +- .../results/clientnegative/strict_join.q.out| 2 +- .../results/clientnegative/strict_join_2.q.out | 2 +- .../results/clientnegative/strict_orderby.q.out | 2 +- .../clientnegative/strict_orderby_2.q.out | 1 + .../results/clientnegative/strict_pruning.q.out | 2 +- .../clientnegative/strict_pruning_2.q.out | 2 +- 21 files changed, 41 insertions(+), 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/4a33ec8f/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 4f2e6d3..99e8457 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1096,11 +1096,14 @@ public class HiveConf extends Configuration { "hive.txn.valid.txns,hive.script.operator.env.blacklist", "Comma separated list of keys from the configuration file not to convert to environment " + "variables when invoking the script operator"), -HIVE_STRICT_CHECKS_LARGE_QUERY("hive.strict.checks.large.query", false, +HIVE_STRICT_CHECKS_ORDERBY_NO_LIMIT("hive.strict.checks.orderby.no.limit", false, "Enabling strict large query checks disallows the following:\n" + " Orderby without limit.\n" + +"Note that this check currently does not consider data size, only the query pattern."), + HIVE_STRICT_CHECKS_NO_PARTITION_FILTER("hive.strict.checks.no.partition.filter", false, +"Enabling strict large query checks disallows the following:\n" + " No partition being picked up for a query against partitioned table.\n" + -"Note that these checks currently do not consider data size, only the query pattern."), +"Note that this check currently does not consider data size, only the query pattern."), HIVE_STRICT_CHECKS_TYPE_SAFETY("hive.strict.checks.type.safety", true, "Enabling strict type safety checks disallows the following:\n" + " Comparing bigints and strings.\n" + @@ -4971,10 +4974,10 @@ public class HiveConf extends Configuration { public static class StrictChecks { private static final String NO_LIMIT_MSG = makeMessage( -"Order by-s without limit", ConfVars.HIVE_STRICT_CHECKS_LARGE_QUERY); +"Order by-s without limit", ConfVars.HIVE_STRICT_CHECKS_ORDERBY_NO_LIMIT); public static final String NO_PARTITIONLESS_MSG = makeMessage( "Queries against partitioned tables without a partition filter", -ConfVars.HIVE_STRICT_CHECKS_LARGE_QUERY); +ConfVars.HIVE_STRICT_CHECKS_NO_PARTITION_FILTER); private static final String NO_COMPARES_MSG = makeMessage( "Unsafe compares between different types", ConfVars.HIVE_STRICT_CHECKS_TYPE_SAFETY); private static final String NO_CARTESIAN_MSG = makeMessage( @@ -4984,17 +4987,17 @@ public class HiveConf extends Configuration { private static String makeMessage(String what, ConfVars setting) { return what + " are disabled for safety reasons. If you know what you are doing, please set "
hive git commit: HIVE-18489: Automatically migrate s3n URIs to s3a URIs (addendum)
Repository: hive Updated Branches: refs/heads/master 4183ea900 -> 6dc245241 HIVE-18489: Automatically migrate s3n URIs to s3a URIs (addendum) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6dc24524 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6dc24524 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6dc24524 Branch: refs/heads/master Commit: 6dc24524127fc165f212de739bfbf8425d6d2226 Parents: 4183ea9 Author: Sahil TakiarAuthored: Wed Jan 24 14:59:23 2018 -0800 Committer: Sahil Takiar Committed: Wed Jan 24 14:59:23 2018 -0800 -- .../postgres/048-HIVE-18489.postgres.sql| 24 ++-- 1 file changed, 12 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/6dc24524/metastore/scripts/upgrade/postgres/048-HIVE-18489.postgres.sql -- diff --git a/metastore/scripts/upgrade/postgres/048-HIVE-18489.postgres.sql b/metastore/scripts/upgrade/postgres/048-HIVE-18489.postgres.sql index fb4d0a2..ddedfe2 100644 --- a/metastore/scripts/upgrade/postgres/048-HIVE-18489.postgres.sql +++ b/metastore/scripts/upgrade/postgres/048-HIVE-18489.postgres.sql @@ -1,23 +1,23 @@ -UPDATE FUNC_RU - SET RESOURCE_URI = 's3a' || SUBSTR(RESOURCE_URI, 4) +UPDATE "FUNC_RU" + SET "RESOURCE_URI" = 's3a' || SUBSTR("RESOURCE_URI", 4) WHERE - RESOURCE_URI LIKE 's3n://%' + "RESOURCE_URI" LIKE 's3n://%' ; -UPDATE SKEWED_COL_VALUE_LOC_MAP - SET LOCATION = 's3a' || SUBSTR(LOCATION, 4) +UPDATE "SKEWED_COL_VALUE_LOC_MAP" + SET "LOCATION" = 's3a' || SUBSTR("LOCATION", 4) WHERE - LOCATION LIKE 's3n://%' + "LOCATION" LIKE 's3n://%' ; -UPDATE SDS - SET LOCATION = 's3a' || SUBSTR(LOCATION, 4) +UPDATE "SDS" + SET "LOCATION" = 's3a' || SUBSTR("LOCATION", 4) WHERE - LOCATION LIKE 's3n://%' + "LOCATION" LIKE 's3n://%' ; -UPDATE DBS - SET DB_LOCATION_URI = 's3a' || SUBSTR(DB_LOCATION_URI, 4) +UPDATE "DBS" + SET "DB_LOCATION_URI" = 's3a' || SUBSTR("DB_LOCATION_URI", 4) WHERE - DB_LOCATION_URI LIKE 's3n://%' + "DB_LOCATION_URI" LIKE 's3n://%' ;
hive git commit: HIVE-18485: Add more unit tests for hive.strict.checks.* properties (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 09ce6cdf3 -> 042698ce2 HIVE-18485: Add more unit tests for hive.strict.checks.* properties (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/042698ce Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/042698ce Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/042698ce Branch: refs/heads/master Commit: 042698ce29310d9557995dacf21d1ee7f2613b34 Parents: 09ce6cd Author: Sahil TakiarAuthored: Wed Jan 24 10:04:40 2018 -0800 Committer: Sahil Takiar Committed: Wed Jan 24 10:05:15 2018 -0800 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 4 ++-- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 11 +- .../clientnegative/alter_view_failure6_2.q | 14 .../clientnegative/compare_double_bigint_2.q| 8 +++ .../clientnegative/compare_string_bigint_2.q| 8 +++ ql/src/test/queries/clientnegative/input4_2.q | 8 +++ .../queries/clientnegative/input_part0_neg_2.q | 6 + .../test/queries/clientnegative/strict_join.q | 5 + .../test/queries/clientnegative/strict_join_2.q | 6 + .../queries/clientnegative/strict_pruning_2.q | 9 .../clientnegative/alter_view_failure6.q.out| 4 ++-- .../clientnegative/alter_view_failure6_2.q.out | 23 .../clientnegative/compare_double_bigint.q.out | 2 +- .../compare_double_bigint_2.q.out | 1 + .../clientnegative/compare_string_bigint.q.out | 2 +- .../compare_string_bigint_2.q.out | 1 + ql/src/test/results/clientnegative/input4.q.out | 2 +- .../test/results/clientnegative/input4_2.q.out | 1 + .../clientnegative/input_part0_neg.q.out| 2 +- .../clientnegative/input_part0_neg_2.q.out | 1 + .../results/clientnegative/strict_join.q.out| 2 +- .../results/clientnegative/strict_join_2.q.out | 1 + .../results/clientnegative/strict_orderby.q.out | 2 +- .../results/clientnegative/strict_pruning.q.out | 2 +- .../clientnegative/strict_pruning_2.q.out | 1 + 25 files changed, 105 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/042698ce/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 858f22b..0c2cf05 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -4961,7 +4961,7 @@ public class HiveConf extends Configuration { private static final String NO_LIMIT_MSG = makeMessage( "Order by-s without limit", ConfVars.HIVE_STRICT_CHECKS_LARGE_QUERY); -private static final String NO_PARTITIONLESS_MSG = makeMessage( +public static final String NO_PARTITIONLESS_MSG = makeMessage( "Queries against partitioned tables without a partition filter", ConfVars.HIVE_STRICT_CHECKS_LARGE_QUERY); private static final String NO_COMPARES_MSG = makeMessage( @@ -4972,7 +4972,7 @@ public class HiveConf extends Configuration { "Load into bucketed tables", ConfVars.HIVE_STRICT_CHECKS_BUCKETING); private static String makeMessage(String what, ConfVars setting) { - return what + " are disabled for safety reasons. If you know what you are doing, please set" + return what + " are disabled for safety reasons. If you know what you are doing, please set " + setting.varname + " to false and that " + ConfVars.HIVEMAPREDMODE.varname + " is not" + " set to 'strict' to proceed. Note that if you may get errors or incorrect results if" + " you make a mistake while using some of the unsafe features."; http://git-wip-us.apache.org/repos/asf/hive/blob/042698ce/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 5baac18..134faee 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -114,15 +114,12 @@ public enum ErrorMsg { CLUSTERBY_ORDERBY_CONFLICT(10050, "Cannot have both CLUSTER BY and ORDER BY clauses"), NO_LIMIT_WITH_ORDERBY(10051, "In strict mode, if ORDER BY is specified, " + "LIMIT must also be specified"), - NO_CARTESIAN_PRODUCT(10052, "In strict mode, cartesian product is not allowed. " - + "If you really want to perform the operation, set hive.mapred.mode=nonstrict"),
hive git commit: HIVE-18489: Automatically migration s3n URIs to s3a URIs (Sahil Takiar, reviewed by Aihua Xu)
Repository: hive Updated Branches: refs/heads/master f0b824ade -> ed256f350 HIVE-18489: Automatically migration s3n URIs to s3a URIs (Sahil Takiar, reviewed by Aihua Xu) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ed256f35 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ed256f35 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ed256f35 Branch: refs/heads/master Commit: ed256f35057b36ec25f806f69db490966ead8d0d Parents: f0b824a Author: Sahil TakiarAuthored: Tue Jan 23 08:30:41 2018 -0800 Committer: Sahil Takiar Committed: Tue Jan 23 08:31:38 2018 -0800 -- .../upgrade/derby/049-HIVE-18489.derby.sql | 23 .../derby/upgrade-2.3.0-to-3.0.0.derby.sql | 1 + .../upgrade/mssql/034-HIVE-18489.mssql.sql | 23 .../mssql/upgrade-2.3.0-to-3.0.0.mssql.sql | 1 + .../upgrade/mysql/049-HIVE-18489.mysql.sql | 23 .../mysql/upgrade-2.3.0-to-3.0.0.mysql.sql | 1 + .../upgrade/oracle/049-HIVE-18489.oracle.sql| 23 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql| 1 + .../postgres/048-HIVE-18489.postgres.sql| 23 .../upgrade-2.3.0-to-3.0.0.postgres.sql | 1 + 10 files changed, 120 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ed256f35/metastore/scripts/upgrade/derby/049-HIVE-18489.derby.sql -- diff --git a/metastore/scripts/upgrade/derby/049-HIVE-18489.derby.sql b/metastore/scripts/upgrade/derby/049-HIVE-18489.derby.sql new file mode 100644 index 000..fb4d0a2 --- /dev/null +++ b/metastore/scripts/upgrade/derby/049-HIVE-18489.derby.sql @@ -0,0 +1,23 @@ +UPDATE FUNC_RU + SET RESOURCE_URI = 's3a' || SUBSTR(RESOURCE_URI, 4) +WHERE + RESOURCE_URI LIKE 's3n://%' +; + +UPDATE SKEWED_COL_VALUE_LOC_MAP + SET LOCATION = 's3a' || SUBSTR(LOCATION, 4) +WHERE + LOCATION LIKE 's3n://%' +; + +UPDATE SDS + SET LOCATION = 's3a' || SUBSTR(LOCATION, 4) +WHERE + LOCATION LIKE 's3n://%' +; + +UPDATE DBS + SET DB_LOCATION_URI = 's3a' || SUBSTR(DB_LOCATION_URI, 4) +WHERE + DB_LOCATION_URI LIKE 's3n://%' +; http://git-wip-us.apache.org/repos/asf/hive/blob/ed256f35/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql -- diff --git a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql index 574ad28..4372498 100644 --- a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql +++ b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql @@ -7,5 +7,6 @@ RUN '045-HIVE-16886.derby.sql'; RUN '046-HIVE-17566.derby.sql'; RUN '047-HIVE-18202.derby.sql'; RUN '048-HIVE-14498.derby.sql'; +RUN '049-HIVE-18489.derby.sql'; UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1; http://git-wip-us.apache.org/repos/asf/hive/blob/ed256f35/metastore/scripts/upgrade/mssql/034-HIVE-18489.mssql.sql -- diff --git a/metastore/scripts/upgrade/mssql/034-HIVE-18489.mssql.sql b/metastore/scripts/upgrade/mssql/034-HIVE-18489.mssql.sql new file mode 100644 index 000..197aa8e --- /dev/null +++ b/metastore/scripts/upgrade/mssql/034-HIVE-18489.mssql.sql @@ -0,0 +1,23 @@ +UPDATE FUNC_RU + SET RESOURCE_URI = 's3a' + SUBSTRING(RESOURCE_URI, 4, LEN(RESOURCE_URI)) +WHERE + RESOURCE_URI LIKE 's3n://%' +; + +UPDATE SKEWED_COL_VALUE_LOC_MAP + SET LOCATION = 's3a' + SUBSTRING(LOCATION, 4, LEN(LOCATION)) +WHERE + LOCATION LIKE 's3n://%' +; + +UPDATE SDS + SET LOCATION = 's3a' + SUBSTRING(LOCATION, 4, LEN(LOCATION)) +WHERE + LOCATION LIKE 's3n://%' +; + +UPDATE DBS + SET DB_LOCATION_URI = 's3a' + SUBSTRING(DB_LOCATION_URI, 4, LEN(DB_LOCATION_URI)) +WHERE + DB_LOCATION_URI LIKE 's3n://%' +; http://git-wip-us.apache.org/repos/asf/hive/blob/ed256f35/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql -- diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql index ebe10ff..52727f3 100644 --- a/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql +++ b/metastore/scripts/upgrade/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql @@ -8,6 +8,7 @@ SELECT 'Upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE; :r 031-HIVE-17566.mssql.sql :r 032-HIVE-18202.mssql.sql :r 033-HIVE-14498.mssql.sql +:r 034-HIVE-18489.mssql.sql UPDATE VERSION SET
hive git commit: HIVE-18214: Flaky test: TestSparkClient (Sahil Takiar, reviewed by Peter Vary)
Repository: hive Updated Branches: refs/heads/master 2402f3ca5 -> 87860fbca HIVE-18214: Flaky test: TestSparkClient (Sahil Takiar, reviewed by Peter Vary) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/87860fbc Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/87860fbc Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/87860fbc Branch: refs/heads/master Commit: 87860fbca42b2477f504c8b1cefd43c865a2629c Parents: 2402f3c Author: Sahil TakiarAuthored: Tue Jan 16 09:18:32 2018 -0800 Committer: Sahil Takiar Committed: Tue Jan 16 09:18:32 2018 -0800 -- .../hive/spark/client/SparkClientFactory.java | 3 - .../hive/spark/client/SparkClientImpl.java | 528 +-- .../hive/spark/client/TestSparkClient.java | 48 +- 3 files changed, 265 insertions(+), 314 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/87860fbc/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java index 50c7bb2..8abeed8 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java @@ -34,9 +34,6 @@ import com.google.common.base.Throwables; @InterfaceAudience.Private public final class SparkClientFactory { - /** Used to run the driver in-process, mostly for testing. */ - static final String CONF_KEY_IN_PROCESS = "spark.client.do_not_use.run_driver_in_process"; - /** Used by client and driver to share a client ID for establishing an RPC session. */ static final String CONF_CLIENT_ID = "spark.client.authentication.client_id"; http://git-wip-us.apache.org/repos/asf/hive/blob/87860fbc/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java index 49b7deb..eed8e53 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java @@ -214,324 +214,294 @@ class SparkClientImpl implements SparkClient { final String serverAddress = rpcServer.getAddress(); final String serverPort = String.valueOf(rpcServer.getPort()); -if (conf.containsKey(SparkClientFactory.CONF_KEY_IN_PROCESS)) { - // Mostly for testing things quickly. Do not do this in production. - // when invoked in-process it inherits the environment variables of the parent - LOG.warn(" Running remote driver in-process. "); - runnable = new Runnable() { -@Override -public void run() { - List args = Lists.newArrayList(); - args.add("--remote-host"); - args.add(serverAddress); - args.add("--remote-port"); - args.add(serverPort); - args.add("--client-id"); - args.add(clientId); - args.add("--secret"); - args.add(secret); - - for (Map.Entry e : conf.entrySet()) { -args.add("--conf"); -args.add(String.format("%s=%s", e.getKey(), conf.get(e.getKey(; - } - try { -RemoteDriver.main(args.toArray(new String[args.size()])); - } catch (Exception e) { -LOG.error("Error running driver.", e); - } -} - }; -} else { - // If a Spark installation is provided, use the spark-submit script. Otherwise, call the - // SparkSubmit class directly, which has some caveats (like having to provide a proper - // version of Guava on the classpath depending on the deploy mode). - String sparkHome = Strings.emptyToNull(conf.get(SPARK_HOME_KEY)); - if (sparkHome == null) { -sparkHome = Strings.emptyToNull(System.getenv(SPARK_HOME_ENV)); - } +// If a Spark installation is provided, use the spark-submit script. Otherwise, call the +// SparkSubmit class directly, which has some caveats (like having to provide a proper +// version of Guava on the classpath depending on the deploy mode). +String sparkHome = Strings.emptyToNull(conf.get(SPARK_HOME_KEY)); +if (sparkHome == null) { + sparkHome = Strings.emptyToNull(System.getenv(SPARK_HOME_ENV)); +} +if (sparkHome == null) { + sparkHome =
hive git commit: HIVE-18311: Enable smb_mapjoin_8.q for cli driver (Janaki Lahorani, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master 77360d6cf -> b34b30039 HIVE-18311: Enable smb_mapjoin_8.q for cli driver (Janaki Lahorani, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b34b3003 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b34b3003 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b34b3003 Branch: refs/heads/master Commit: b34b300399f3b684ec8b28b78de839b2116546c3 Parents: 77360d6 Author: Janaki LahoraniAuthored: Thu Jan 4 10:34:35 2018 -0800 Committer: Sahil Takiar Committed: Thu Jan 4 10:34:54 2018 -0800 -- .../test/resources/testconfiguration.properties | 1 - .../results/clientpositive/smb_mapjoin_8.q.out | 464 +++ 2 files changed, 464 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/b34b3003/itests/src/test/resources/testconfiguration.properties -- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 9121ca8..98e390c 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -34,7 +34,6 @@ disabled.query.files=ql_rewrite_gbtoidx.q,\ orc_llap.q,\ ql_rewrite_gbtoidx_cbo_2.q,\ rcfile_merge1.q,\ - smb_mapjoin_8.q,\ stats_filemetadata.q,\ mm_exim.q http://git-wip-us.apache.org/repos/asf/hive/blob/b34b3003/ql/src/test/results/clientpositive/smb_mapjoin_8.q.out -- diff --git a/ql/src/test/results/clientpositive/smb_mapjoin_8.q.out b/ql/src/test/results/clientpositive/smb_mapjoin_8.q.out new file mode 100644 index 000..20cfc0d --- /dev/null +++ b/ql/src/test/results/clientpositive/smb_mapjoin_8.q.out @@ -0,0 +1,464 @@ +PREHOOK: query: create table smb_bucket_input (key int, value string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket_input +POSTHOOK: query: create table smb_bucket_input (key int, value string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket_input +PREHOOK: query: load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input +PREHOOK: type: LOAD + A masked pattern was here +PREHOOK: Output: default@smb_bucket_input +POSTHOOK: query: load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input +POSTHOOK: type: LOAD + A masked pattern was here +POSTHOOK: Output: default@smb_bucket_input +PREHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket4_2 +PREHOOK: query: CREATE TABLE smb_bucket4_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@smb_bucket4_3 +POSTHOOK: query: CREATE TABLE smb_bucket4_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@smb_bucket4_3 +PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4 or key=2000 or key=4000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_input +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_input where key=4 or key=2000 or key=4000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_input +POSTHOOK: Output: default@smb_bucket4_1 +POSTHOOK: Lineage: smb_bucket4_1.key SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: smb_bucket4_1.value SIMPLE [(smb_bucket_input)smb_bucket_input.FieldSchema(name:value, type:string,
hive git commit: HIVE-17929: Use sessionId for HoS Remote Driver Client id (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master ed99182a2 -> d0156565a HIVE-17929: Use sessionId for HoS Remote Driver Client id (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d0156565 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d0156565 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d0156565 Branch: refs/heads/master Commit: d0156565adec240735850bedc7729fa43cd05460 Parents: ed99182 Author: Sahil TakiarAuthored: Wed Jan 3 11:46:53 2018 -0800 Committer: Sahil Takiar Committed: Wed Jan 3 11:46:53 2018 -0800 -- .../hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java| 2 +- .../hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java | 6 -- .../org/apache/hive/spark/client/SparkClientFactory.java | 4 ++-- .../java/org/apache/hive/spark/client/SparkClientImpl.java | 8 .../java/org/apache/hive/spark/client/TestSparkClient.java | 3 ++- 5 files changed, 13 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/d0156565/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index bdb0798..f9e9328 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -73,7 +73,7 @@ public class HiveSparkClientFactory { // With local spark context, all user sessions share the same spark context. return LocalHiveSparkClient.getInstance(generateSparkConf(sparkConf), hiveconf); } else { - return new RemoteHiveSparkClient(hiveconf, sparkConf); + return new RemoteHiveSparkClient(hiveconf, sparkConf, sessionId); } } http://git-wip-us.apache.org/repos/asf/hive/blob/d0156565/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 9b38c1a..d9d4b37 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -88,18 +88,20 @@ public class RemoteHiveSparkClient implements HiveSparkClient { private transient List localFiles = new ArrayList(); private final transient long sparkClientTimtout; + private final String sessionId; - RemoteHiveSparkClient(HiveConf hiveConf, Map conf) throws Exception { + RemoteHiveSparkClient(HiveConf hiveConf, Map conf, String sessionId) throws Exception { this.hiveConf = hiveConf; sparkClientTimtout = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT, TimeUnit.SECONDS); sparkConf = HiveSparkClientFactory.generateSparkConf(conf); this.conf = conf; +this.sessionId = sessionId; createRemoteClient(); } private void createRemoteClient() throws Exception { -remoteClient = SparkClientFactory.createClient(conf, hiveConf); +remoteClient = SparkClientFactory.createClient(conf, hiveConf, sessionId); if (HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_PREWARM_ENABLED) && (SparkClientUtilities.isYarnMaster(hiveConf.get("spark.master")) || http://git-wip-us.apache.org/repos/asf/hive/blob/d0156565/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java -- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java index 8cedd30..50c7bb2 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java @@ -83,10 +83,10 @@ public final class SparkClientFactory { * @param sparkConf Configuration for the remote Spark application, contains spark.* properties. * @param hiveConf Configuration for Hive, contains hive.* properties. */ - public static SparkClient createClient(Map sparkConf, HiveConf hiveConf) + public static SparkClient createClient(Map sparkConf, HiveConf hiveConf, String sessionId) throws IOException, SparkException {
hive git commit: HIVE-18255: spark-client jar should be prefixed with hive- (Sahil Takiar, reviewed by Rui Li)
Repository: hive Updated Branches: refs/heads/master 98c95c67a -> ed99182a2 HIVE-18255: spark-client jar should be prefixed with hive- (Sahil Takiar, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ed99182a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ed99182a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ed99182a Branch: refs/heads/master Commit: ed99182a274e7f7075c8746d7a85b60b5199339d Parents: 98c95c6 Author: Sahil TakiarAuthored: Wed Jan 3 10:01:01 2018 -0800 Committer: Sahil Takiar Committed: Wed Jan 3 10:01:30 2018 -0800 -- ql/pom.xml | 4 ++-- spark-client/pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/ed99182a/ql/pom.xml -- diff --git a/ql/pom.xml b/ql/pom.xml index cbf71cd..7019a91 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -84,7 +84,7 @@ org.apache.hive - spark-client + hive-spark-client ${project.version} @@ -910,7 +910,7 @@ org.codehaus.jackson:jackson-mapper-asl com.google.guava:guava net.sf.opencsv:opencsv - org.apache.hive:spark-client + org.apache.hive:hive-spark-client org.apache.hive:hive-storage-api org.apache.orc:orc-core org.apache.orc:orc-tools http://git-wip-us.apache.org/repos/asf/hive/blob/ed99182a/spark-client/pom.xml -- diff --git a/spark-client/pom.xml b/spark-client/pom.xml index b38c6fa..de6b018 100644 --- a/spark-client/pom.xml +++ b/spark-client/pom.xml @@ -26,9 +26,9 @@ org.apache.hive - spark-client + hive-spark-client jar - Spark Remote Client + Hive Spark Remote Client 3.0.0-SNAPSHOT
hive git commit: HIVE-18248: Clean up parameters (Janaki Lahorani, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master a6fab1438 -> 4ec47a6e6 HIVE-18248: Clean up parameters (Janaki Lahorani, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4ec47a6e Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4ec47a6e Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4ec47a6e Branch: refs/heads/master Commit: 4ec47a6e6ffb6a070295348308e3a8bcbc246190 Parents: a6fab14 Author: Janaki LahoraniAuthored: Tue Dec 19 17:11:20 2017 -0800 Committer: Sahil Takiar Committed: Tue Dec 19 17:26:46 2017 -0800 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 9 +- .../apache/hive/jdbc/TestRestrictedList.java| 192 +++ .../hadoop/hive/ql/session/SessionState.java| 7 + .../change_hive_hdfs_session_path.q | 2 + .../change_hive_local_session_path.q| 2 + .../change_hive_tmp_table_space.q | 2 + .../change_hive_hdfs_session_path.q.out | 2 + .../change_hive_local_session_path.q.out| 2 + .../change_hive_tmp_table_space.q.out | 2 + 9 files changed, 219 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/4ec47a6e/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 28b52d4..8648a38 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3582,6 +3582,7 @@ public class HiveConf extends Configuration { HIVE_MM_AVOID_GLOBSTATUS_ON_S3("hive.mm.avoid.s3.globstatus", true, "Whether to use listFiles (optimized on S3) instead of globStatus when on S3."), +// If a parameter is added to the restricted list, add a test in TestRestrictedList.Java HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list", "hive.security.authenticator.manager,hive.security.authorization.manager," + "hive.security.metastore.authorization.manager,hive.security.metastore.authenticator.manager," + @@ -3610,7 +3611,9 @@ public class HiveConf extends Configuration { "bonecp.,"+ "hive.druid.broker.address.default,"+ "hive.druid.coordinator.address.default,"+ -"hikari.", +"hikari.,"+ +"hadoop.bin.path,"+ +"yarn.bin.path", "Comma separated list of configuration options which are immutable at runtime"), HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list", METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname @@ -4752,6 +4755,10 @@ public class HiveConf extends Configuration { hiveSiteURL = location; } + public static void setHivemetastoreSiteUrl(URL location) { +hivemetastoreSiteUrl = location; + } + public static URL getHiveSiteLocation() { return hiveSiteURL; } http://git-wip-us.apache.org/repos/asf/hive/blob/4ec47a6e/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java new file mode 100644 index 000..1ec77b0 --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.jdbc; + +import java.io.File; +import java.net.URL; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.HashSet; + +import org.apache.hadoop.hive.conf.HiveConf; +import
hive git commit: HIVE-18054: Make Lineage work with concurrent queries on a Session (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master f52e8b4ba -> 646ccce8e HIVE-18054: Make Lineage work with concurrent queries on a Session (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/646ccce8 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/646ccce8 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/646ccce8 Branch: refs/heads/master Commit: 646ccce8ea3e8c944be164f86dbd5d3428bdbc44 Parents: f52e8b4 Author: Andrew ShermanAuthored: Sat Dec 16 15:14:54 2017 -0600 Committer: Sahil Takiar Committed: Sat Dec 16 15:24:11 2017 -0600 -- .../java/org/apache/hive/jdbc/ReadableHook.java | 52 + .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 114 +++ .../java/org/apache/hadoop/hive/ql/Driver.java | 46 ++-- .../org/apache/hadoop/hive/ql/QueryState.java | 35 +- .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 6 +- .../apache/hadoop/hive/ql/exec/MoveTask.java| 9 +- .../org/apache/hadoop/hive/ql/exec/Task.java| 4 + .../bootstrap/load/table/LoadPartitions.java| 3 +- .../repl/bootstrap/load/table/LoadTable.java| 3 +- .../hadoop/hive/ql/hooks/HookContext.java | 9 +- .../hadoop/hive/ql/hooks/LineageLogger.java | 56 +++-- .../hive/ql/index/AggregateIndexHandler.java| 7 +- .../hadoop/hive/ql/index/HiveIndexHandler.java | 6 +- .../hive/ql/index/TableBasedIndexHandler.java | 18 ++- .../ql/index/bitmap/BitmapIndexHandler.java | 8 +- .../ql/index/compact/CompactIndexHandler.java | 8 +- .../hive/ql/optimizer/GenMRFileSink1.java | 2 +- .../hive/ql/optimizer/GenMapRedUtils.java | 27 +++-- .../hadoop/hive/ql/optimizer/IndexUtils.java| 6 +- .../hive/ql/optimizer/lineage/Generator.java| 8 +- .../hive/ql/parse/DDLSemanticAnalyzer.java | 15 ++- .../hive/ql/parse/ExplainSemanticAnalyzer.java | 2 +- .../hadoop/hive/ql/parse/GenTezUtils.java | 3 +- .../hive/ql/parse/ImportSemanticAnalyzer.java | 5 +- .../hadoop/hive/ql/parse/IndexUpdater.java | 9 +- .../hive/ql/parse/LoadSemanticAnalyzer.java | 2 +- .../ql/parse/ReplicationSemanticAnalyzer.java | 4 +- .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 8 +- .../hadoop/hive/ql/parse/TaskCompiler.java | 11 +- .../hive/ql/parse/spark/GenSparkUtils.java | 2 +- .../apache/hadoop/hive/ql/plan/MoveWork.java| 25 +--- .../hadoop/hive/ql/session/LineageState.java| 2 +- .../hadoop/hive/ql/session/SessionState.java| 15 --- ...TestGenMapRedUtilsCreateConditionalTask.java | 18 ++- 34 files changed, 403 insertions(+), 145 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/646ccce8/itests/hive-unit/src/test/java/org/apache/hive/jdbc/ReadableHook.java -- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/ReadableHook.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/ReadableHook.java new file mode 100644 index 000..2dd283f --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/ReadableHook.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.jdbc; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext; +import org.apache.hadoop.hive.ql.hooks.HookContext; + +/** + * An ExecuteWithHookContext that stores HookContexts in memory and makes them available for reading + */ +public class ReadableHook implements ExecuteWithHookContext { + + private static List hookList = Collections.synchronizedList(new ArrayList<>()); + + @Override + public void run(HookContext hookContext) throws Exception { +hookList.add(hookContext); + } + + /** + * @return the stored HookContexts. + */ + public static List getHookList() { +return hookList; +
hive git commit: HIVE-18166: Result of hive.query.string is encoded. (Janaki Lahorani, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master b23c3299f -> 7ddd915bf HIVE-18166: Result of hive.query.string is encoded. (Janaki Lahorani, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7ddd915b Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7ddd915b Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7ddd915b Branch: refs/heads/master Commit: 7ddd915bf82a68c8ab73b0c4ca409f1a6d43d227 Parents: b23c329 Author: Janaki LahoraniAuthored: Wed Dec 6 22:45:06 2017 -0800 Committer: Sahil Takiar Committed: Wed Dec 6 22:45:06 2017 -0800 -- .../org/apache/hadoop/hive/conf/HiveConf.java | 17 + .../hadoop/hive/ql/processors/SetProcessor.java | 7 +- .../queries/clientpositive/testSetQueryString.q | 4 +++ .../clientpositive/testSetQueryString.q.out | 26 4 files changed, 53 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/7ddd915b/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 75b7707..8b2b333 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -329,6 +329,13 @@ public class HiveConf extends Configuration { }; /** + * encoded parameter values are ;-) encoded. Use decoder to get ;-) decoded string + */ + public static final HiveConf.ConfVars[] ENCODED_CONF = { + ConfVars.HIVEQUERYSTRING + }; + + /** * Variables used by LLAP daemons. * TODO: Eventually auto-populate this based on prefixes. The conf variables * will need to be renamed for this. @@ -3933,6 +3940,16 @@ public class HiveConf extends Configuration { return Iterables.any(hiddenSet, hiddenVar -> name.startsWith(hiddenVar)); } + public static boolean isEncodedPar(String name) { +for (ConfVars confVar : HiveConf.ENCODED_CONF) { + ConfVars confVar1 = confVar; + if (confVar1.varname.equals(name)) { +return true; + } +} +return false; + } + /** * check whether spark related property is updated, which includes spark configurations, * RSC configurations and yarn configuration in Spark on YARN mode. http://git-wip-us.apache.org/repos/asf/hive/blob/7ddd915b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java -- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java index b9ca938..64c4711 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java @@ -143,7 +143,12 @@ public class SetProcessor implements CommandProcessor { if (ss.getConf().isHiddenConfig(s)) { ss.out.println(s + " is a hidden config"); } else if (ss.getConf().get(s) != null) { - ss.out.println(s + "=" + ss.getConf().get(s)); + if (ss.getConf().isEncodedPar(s)) { +ss.out.println(s + "=" + HiveConf.EncoderDecoderFactory.URL_ENCODER_DECODER +.decode(ss.getConf().get(s))); + } else { +ss.out.println(s + "=" + ss.getConf().get(s)); + } } else if (ss.getHiveVariables().containsKey(s)) { ss.out.println(s + "=" + ss.getHiveVariables().get(s)); } else { http://git-wip-us.apache.org/repos/asf/hive/blob/7ddd915b/ql/src/test/queries/clientpositive/testSetQueryString.q -- diff --git a/ql/src/test/queries/clientpositive/testSetQueryString.q b/ql/src/test/queries/clientpositive/testSetQueryString.q new file mode 100644 index 000..f11afb2 --- /dev/null +++ b/ql/src/test/queries/clientpositive/testSetQueryString.q @@ -0,0 +1,4 @@ +create table t1 (c1 int); +insert into t1 values (1); +select * from t1; +set hive.query.string; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/7ddd915b/ql/src/test/results/clientpositive/testSetQueryString.q.out -- diff --git a/ql/src/test/results/clientpositive/testSetQueryString.q.out b/ql/src/test/results/clientpositive/testSetQueryString.q.out new file mode 100644 index 000..0bf1919 --- /dev/null +++ b/ql/src/test/results/clientpositive/testSetQueryString.q.out @@ -0,0 +1,26 @@ +PREHOOK: query: create table t1 (c1 int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK:
hive git commit: HIVE-18127: Do not strip '--' comments from shell commands issued from CliDriver (Andrew Sherman, reviewed by Sahil Takiar)
Repository: hive Updated Branches: refs/heads/master f68ebdce7 -> 33d527f25 HIVE-18127: Do not strip '--' comments from shell commands issued from CliDriver (Andrew Sherman, reviewed by Sahil Takiar) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/33d527f2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/33d527f2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/33d527f2 Branch: refs/heads/master Commit: 33d527f257137c421e8d362c5cf15d8e8fe26599 Parents: f68ebdc Author: Andrew ShermanAuthored: Fri Dec 1 09:39:51 2017 -0800 Committer: Sahil Takiar Committed: Fri Dec 1 09:40:16 2017 -0800 -- .../apache/hive/beeline/cli/TestHiveCli.java| 7 .../org/apache/hadoop/hive/cli/CliDriver.java | 4 +- .../hadoop/hive/cli/TestCliDriverMethods.java | 43 .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 13 ++ 4 files changed, 65 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/33d527f2/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java -- diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java index 28b0cf9..068bb8d 100644 --- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java +++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java @@ -112,6 +112,13 @@ public class TestHiveCli { } @Test + public void testCommentStripping() { +// this should work as comments are stripped by HiveCli +verifyCMD("!ls --abcdefghijklmnopqrstuvwxyz\n", "src", os, null, ERRNO_OK, true); + } + + + @Test public void testSetPromptValue() { verifyCMD("set hive.cli.prompt=MYCLI;SHOW\nTABLES;", "MYCLI> ", errS, null, ERRNO_OK, true); http://git-wip-us.apache.org/repos/asf/hive/blob/33d527f2/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java -- diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java index cf06582..bd0b422 100644 --- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java +++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java @@ -158,8 +158,8 @@ public class CliDriver { } } } else if (cmd_trimmed.startsWith("!")) { - - String shell_cmd = cmd_trimmed.substring(1); + // for shell commands, use unstripped command + String shell_cmd = cmd.trim().substring(1); shell_cmd = new VariableSubstitution(new HiveVariableSource() { @Override public Map getHiveVariable() { http://git-wip-us.apache.org/repos/asf/hive/blob/33d527f2/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java -- diff --git a/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java b/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java index bf23ba3..8f1c15e 100644 --- a/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java +++ b/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java @@ -104,6 +104,49 @@ public class TestCliDriverMethods extends TestCase { verify(mockOut, never()).print(anyString()); } + // Test that CliDriver does not strip comments starting with '--' + public void testThatCliDriverDoesNotStripComments() throws Exception { +// We need to overwrite System.out and System.err as that is what is used in ShellCmdExecutor +// So save old values... +PrintStream oldOut = System.out; +PrintStream oldErr = System.err; + +// Capture stdout and stderr +ByteArrayOutputStream dataOut = new ByteArrayOutputStream(); +PrintStream out = new PrintStream(dataOut); +System.setOut(out); +ByteArrayOutputStream dataErr = new ByteArrayOutputStream(); +PrintStream err = new PrintStream(dataErr); +System.setErr(err); + +CliSessionState ss = new CliSessionState(new HiveConf()); +ss.out = out; +ss.err = err; + +// Save output as yo cannot print it while System.out and System.err are weird +String message; +String errors; +int ret; +try { + CliSessionState.start(ss); + CliDriver cliDriver = new CliDriver(); + // issue a command with bad options + ret = cliDriver.processCmd("!ls --abcdefghijklmnopqrstuvwxyz123456789"); +} finally { + // restore System.out and System.err + System.setOut(oldOut); + System.setErr(oldErr); +} +message = dataOut.toString("UTF-8"); +errors = dataErr.toString("UTF-8"); +assertTrue("Comments with '--; should
svn commit: r1020872 - in /websites/production/hive/content: ./ javadocs/
Author: stakiar Date: Wed Nov 15 04:06:44 2017 New Revision: 1020872 Log: Release 2.3.2 Added: websites/production/hive/content/ - copied from r1020871, websites/staging/hive/trunk/content/ websites/production/hive/content/javadocs/ - copied from r1020871, websites/production/hive/content/javadocs/
hive git commit: Preparing for 2.3.3 development
Repository: hive Updated Branches: refs/heads/branch-2.3 857a9fd8a -> 2fa7a6344 Preparing for 2.3.3 development Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2fa7a634 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2fa7a634 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2fa7a634 Branch: refs/heads/branch-2.3 Commit: 2fa7a6344bf323e2e8d730896871d775e6c103fd Parents: 857a9fd Author: Sahil TakiarAuthored: Tue Nov 14 14:35:41 2017 -0800 Committer: Sahil Takiar Committed: Tue Nov 14 14:35:41 2017 -0800 -- accumulo-handler/pom.xml | 2 +- beeline/pom.xml | 2 +- cli/pom.xml | 2 +- common/pom.xml | 2 +- contrib/pom.xml | 2 +- druid-handler/pom.xml| 2 +- hbase-handler/pom.xml| 2 +- hcatalog/core/pom.xml| 2 +- hcatalog/hcatalog-pig-adapter/pom.xml| 2 +- hcatalog/pom.xml | 2 +- hcatalog/server-extensions/pom.xml | 2 +- hcatalog/streaming/pom.xml | 2 +- hcatalog/webhcat/java-client/pom.xml | 2 +- hcatalog/webhcat/svr/pom.xml | 2 +- hplsql/pom.xml | 2 +- itests/custom-serde/pom.xml | 2 +- itests/custom-udfs/pom.xml | 2 +- itests/custom-udfs/udf-classloader-udf1/pom.xml | 2 +- itests/custom-udfs/udf-classloader-udf2/pom.xml | 2 +- itests/custom-udfs/udf-classloader-util/pom.xml | 2 +- itests/custom-udfs/udf-vectorized-badexample/pom.xml | 2 +- itests/hcatalog-unit/pom.xml | 2 +- itests/hive-blobstore/pom.xml| 2 +- itests/hive-jmh/pom.xml | 2 +- itests/hive-minikdc/pom.xml | 2 +- itests/hive-unit-hadoop2/pom.xml | 2 +- itests/hive-unit/pom.xml | 2 +- itests/pom.xml | 2 +- itests/qtest-accumulo/pom.xml| 2 +- itests/qtest-spark/pom.xml | 2 +- itests/qtest/pom.xml | 2 +- itests/test-serde/pom.xml| 2 +- itests/util/pom.xml | 2 +- jdbc-handler/pom.xml | 2 +- jdbc/pom.xml | 2 +- llap-client/pom.xml | 2 +- llap-common/pom.xml | 2 +- llap-ext-client/pom.xml | 2 +- llap-server/pom.xml | 2 +- llap-tez/pom.xml | 2 +- metastore/pom.xml| 2 +- packaging/pom.xml| 2 +- pom.xml | 2 +- ql/pom.xml | 2 +- serde/pom.xml| 2 +- service-rpc/pom.xml | 2 +- service/pom.xml | 2 +- shims/0.23/pom.xml | 2 +- shims/aggregator/pom.xml | 2 +- shims/common/pom.xml | 2 +- shims/pom.xml| 2 +- shims/scheduler/pom.xml | 2 +- spark-client/pom.xml | 4 ++-- testutils/pom.xml| 2 +- vector-code-gen/pom.xml | 2 +- 55 files changed, 56 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hive/blob/2fa7a634/accumulo-handler/pom.xml -- diff --git a/accumulo-handler/pom.xml b/accumulo-handler/pom.xml index 143a901..d96226e 100644 --- a/accumulo-handler/pom.xml +++ b/accumulo-handler/pom.xml @@ -19,7 +19,7 @@ org.apache.hive hive -2.3.2 +2.3.3-SNAPSHOT ../pom.xml http://git-wip-us.apache.org/repos/asf/hive/blob/2fa7a634/beeline/pom.xml -- diff --git a/beeline/pom.xml b/beeline/pom.xml index 12337b2..0b2729c 100644 --- a/beeline/pom.xml +++ b/beeline/pom.xml @@ -19,7 +19,7 @@ org.apache.hive hive -2.3.2 +2.3.3-SNAPSHOT ../pom.xml
svn commit: r1020863 - in /websites/production/hive/content/javadocs: r2.3.1/ r2.3.2/ r2.3.2/api/ r2.3.2/api/org/ r2.3.2/api/org/apache/ r2.3.2/api/org/apache/hadoop/ r2.3.2/api/org/apache/hadoop/fs/
Author: stakiar Date: Tue Nov 14 22:11:21 2017 New Revision: 1020863 Log: Release 2.3.2 [This commit notification would consist of 4077 parts, which exceeds the limit of 50 ones, so it was shortened to the summary.]
svn commit: r1815273 - in /hive/cms/trunk/content: downloads.mdtext javadoc.mdtext
Author: stakiar Date: Tue Nov 14 22:18:25 2017 New Revision: 1815273 URL: http://svn.apache.org/viewvc?rev=1815273=rev Log: Release 2.3.2 Modified: hive/cms/trunk/content/downloads.mdtext hive/cms/trunk/content/javadoc.mdtext Modified: hive/cms/trunk/content/downloads.mdtext URL: http://svn.apache.org/viewvc/hive/cms/trunk/content/downloads.mdtext?rev=1815273=1815272=1815273=diff == --- hive/cms/trunk/content/downloads.mdtext (original) +++ hive/cms/trunk/content/downloads.mdtext Tue Nov 14 22:18:25 2017 @@ -11,6 +11,10 @@ directory. ## News +### 18 November 2017 : release 2.3.2 available +This release works with Hadoop 2.x.y +You can look at the complete [JIRA change log for this release][HIVE_2_3_2_CL]. + ### 24 October 2017 : release 2.3.1 available This release works with Hadoop 2.x.y You can look at the complete [JIRA change log for this release][HIVE_2_3_1_CL]. @@ -116,6 +120,7 @@ This release works with Hadoop 0.20.x, You can look at the complete [JIRA change log for this release][HIVE_10_CL]. [HIVE_DL]: http://www.apache.org/dyn/closer.cgi/hive/ +[HIVE_2_3_2_CL]: https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12342053=Text=12310843 [HIVE_2_3_1_CL]: https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12341418=Text=12310843 [HIVE_2_2_0_CL]: https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12335837=Text=12310843 [HIVE_2_3_0_CL]: https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12340269=Text=12310843 Modified: hive/cms/trunk/content/javadoc.mdtext URL: http://svn.apache.org/viewvc/hive/cms/trunk/content/javadoc.mdtext?rev=1815273=1815272=1815273=diff == --- hive/cms/trunk/content/javadoc.mdtext (original) +++ hive/cms/trunk/content/javadoc.mdtext Tue Nov 14 22:18:25 2017 @@ -2,7 +2,7 @@ Title: Javadoc Recent versions: - * [Hive 2.3.1 Javadocs][r2.3.1] + * [Hive 2.3.2 Javadocs][r2.3.2] * [Hive 2.2.0 Javadocs][r2.2.0] * [Hive 2.1.1 Javadocs][r2.1.1] * [Hive 1.2.2 Javadocs][r1.2.2] @@ -16,6 +16,7 @@ Archived versions: javadoc and sources jars for use in an IDE are also available via [Nexus][] +[r2.3.2]: /javadocs/r2.3.2/api/index.html [r2.3.1]: /javadocs/r2.3.1/api/index.html [r2.2.0]: /javadocs/r2.2.0/api/index.html [r2.1.1]: /javadocs/r2.1.1/api/index.html
[hive] Git Push Summary
Repository: hive Updated Tags: refs/tags/release-2.3.2 [deleted] a0988d39a
[hive] Git Push Summary
Repository: hive Updated Tags: refs/tags/rel/release-2.3.2 [created] 7d66cfc27
[hive] Git Push Summary
Repository: hive Updated Tags: refs/tags/release-2.3.2 [created] a0988d39a